From cc854f408fd06553de4780b1874992c16b71f26e Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Wed, 10 Jul 2024 11:07:30 -0700 Subject: [PATCH 01/66] WordBreakSpellChecker.suggestWordBreaks now does a breadth first search (#12100) --- lucene/CHANGES.txt | 3 + .../search/spell/WordBreakSpellChecker.java | 92 +++++++++++++------ .../spell/TestWordBreakSpellChecker.java | 55 +++++++++-- 3 files changed, 114 insertions(+), 36 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 26a7c06e483..d67318da678 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -282,6 +282,9 @@ Optimizations * GITHUB#13538: Slightly reduce heap usage for HNSW and scalar quantized vector writers. (Ben Trent) +* GITHUB#12100: WordBreakSpellChecker.suggestWordBreaks now does a breadth first search, allowing it to return + better matches with fewer evaluations (hossman) + Changes in runtime behavior --------------------- diff --git a/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java b/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java index b5572f717d6..b3a84e9aaf8 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java @@ -22,6 +22,9 @@ import java.util.PriorityQueue; import java.util.Queue; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.BitSetIterator; +import org.apache.lucene.util.FixedBitSet; /** * A spell checker whose sole function is to offer suggestions by combining multiple terms into one @@ -246,51 +249,80 @@ public class WordBreakSpellChecker { int totalEvaluations, BreakSuggestionSortMethod sortMethod) throws IOException { - String termText = term.text(); - int termLength = termText.codePointCount(0, termText.length()); - int useMinBreakWordLength = minBreakWordLength; - if (useMinBreakWordLength < 1) { - useMinBreakWordLength = 1; - } + final String termText = term.text(); + final int termLength = termText.codePointCount(0, termText.length()); + final int useMinBreakWordLength = Math.max(minBreakWordLength, 1); if (termLength < (useMinBreakWordLength * 2)) { return totalEvaluations; } - for (int i = useMinBreakWordLength; i <= (termLength - useMinBreakWordLength); i++) { + // Two phase breadth first search. + // + // Phase #1: checking for bi-sects of our termText, and recording the + // positions of valid leftText splits for later + + final int maxSplitPosition = termLength - useMinBreakWordLength; + final BitSet validLeftSplitPositions = new FixedBitSet(termText.length()); + + for (int i = useMinBreakWordLength; i <= maxSplitPosition; i++) { if (totalEvaluations >= maxEvaluations) { - break; + return totalEvaluations; } totalEvaluations++; - int end = termText.offsetByCodePoints(0, i); - String leftText = termText.substring(0, end); - String rightText = termText.substring(end); - SuggestWord leftWord = generateSuggestWord(ir, term.field(), leftText); + final int end = termText.offsetByCodePoints(0, i); + final String leftText = termText.substring(0, end); + final String rightText = termText.substring(end); + + final SuggestWord leftWord = generateSuggestWord(ir, term.field(), leftText); if (leftWord.freq >= useMinSuggestionFrequency) { - SuggestWord rightWord = generateSuggestWord(ir, term.field(), rightText); + validLeftSplitPositions.set(end); + final SuggestWord rightWord = generateSuggestWord(ir, term.field(), rightText); if (rightWord.freq >= useMinSuggestionFrequency) { - SuggestWordArrayWrapper suggestion = - new SuggestWordArrayWrapper(newSuggestion(prefix, leftWord, rightWord)); - suggestions.offer(suggestion); + suggestions.offer( + new SuggestWordArrayWrapper(newSuggestion(prefix, leftWord, rightWord))); if (suggestions.size() > maxSuggestions) { suggestions.poll(); } } - int newNumberBreaks = numberBreaks + 1; - if (newNumberBreaks <= maxChanges) { - totalEvaluations = - generateBreakUpSuggestions( - new Term(term.field(), rightWord.string), - ir, - newNumberBreaks, - maxSuggestions, - useMinSuggestionFrequency, - newPrefix(prefix, leftWord), - suggestions, - totalEvaluations, - sortMethod); - } + } + } + + // if we are about to exceed our maxChanges *OR* we have a full list of + // suggestions, we can return now. + // + // (because any subsequent suggestions are garunteed to have more changes + // then anything currently in the queue, and not be competitive) + + final int newNumberBreaks = numberBreaks + 1; + if (totalEvaluations >= maxEvaluations + || newNumberBreaks > maxChanges + || suggestions.size() >= maxSuggestions) { + return totalEvaluations; + } + + // Phase #2: recursing on the right side of any valid leftText terms + final BitSetIterator leftIter = new BitSetIterator(validLeftSplitPositions, 0); + for (int pos = leftIter.nextDoc(); + pos != BitSetIterator.NO_MORE_DOCS; + pos = leftIter.nextDoc()) { + final String leftText = termText.substring(0, pos); + final String rightText = termText.substring(pos); + final SuggestWord leftWord = generateSuggestWord(ir, term.field(), leftText); + totalEvaluations = + generateBreakUpSuggestions( + new Term(term.field(), rightText), + ir, + newNumberBreaks, + maxSuggestions, + useMinSuggestionFrequency, + newPrefix(prefix, leftWord), + suggestions, + totalEvaluations, + sortMethod); + if (totalEvaluations >= maxEvaluations) { + break; } } diff --git a/lucene/suggest/src/test/org/apache/lucene/search/spell/TestWordBreakSpellChecker.java b/lucene/suggest/src/test/org/apache/lucene/search/spell/TestWordBreakSpellChecker.java index 83b7066fa0b..38121ced691 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/spell/TestWordBreakSpellChecker.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/spell/TestWordBreakSpellChecker.java @@ -89,20 +89,24 @@ public class TestWordBreakSpellChecker extends LuceneTestCase { } public void testMaxEvaluations() throws Exception { - final int maxEvals = 100; - try (IndexReader ir = DirectoryReader.open(dir)) { + + final String input = "ab".repeat(5); + final int maxEvals = 100; + final int maxSuggestions = maxEvals * 2; // plenty + WordBreakSpellChecker wbsp = new WordBreakSpellChecker(); - wbsp.setMaxChanges(10); wbsp.setMinBreakWordLength(1); wbsp.setMinSuggestionFrequency(1); - wbsp.setMaxEvaluations(100); - Term term = new Term("abba", "ab".repeat(5)); + wbsp.setMaxChanges(2 * input.length()); // plenty + wbsp.setMaxEvaluations(maxEvals); + + Term term = new Term("abba", input); SuggestWord[][] sw = wbsp.suggestWordBreaks( term, - 500, + maxSuggestions, ir, SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX, BreakSuggestionSortMethod.NUM_CHANGES_THEN_MAX_FREQUENCY); @@ -115,6 +119,45 @@ public class TestWordBreakSpellChecker extends LuceneTestCase { } } + public void testSmallMaxEvaluations() throws Exception { + // even using small maxEvals (relative to maxChanges) should produce + // good results if possible + + try (IndexReader ir = DirectoryReader.open(dir)) { + + final int maxEvals = TestUtil.nextInt(random(), 6, 20); + final int maxSuggestions = maxEvals * 10; // plenty + + WordBreakSpellChecker wbsp = new WordBreakSpellChecker(); + wbsp.setMinBreakWordLength(1); + wbsp.setMinSuggestionFrequency(1); + + wbsp.setMaxChanges(maxEvals * 10); // plenty + wbsp.setMaxEvaluations(maxEvals); + + Term term = new Term("abba", "ababab"); + SuggestWord[][] sw = + wbsp.suggestWordBreaks( + term, + maxSuggestions, + ir, + SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX, + BreakSuggestionSortMethod.NUM_CHANGES_THEN_MAX_FREQUENCY); + + // sanity check that our suggester isn't completely broken + MatcherAssert.assertThat(sw.length, greaterThan(0)); + + // if maxEvaluations is respected, we can't possibly have more suggestions than that + MatcherAssert.assertThat(sw.length, lessThan(maxEvals)); + + // we should have been able to find this "optimal" (due to fewest num changes) + // suggestion before hitting our small maxEvals (and before any suggests with more breaks) + assertEquals(2, sw[0].length); + assertEquals("aba", sw[0][0].string); + assertEquals("bab", sw[0][1].string); + } + } + public void testCombiningWords() throws Exception { IndexReader ir = DirectoryReader.open(dir); WordBreakSpellChecker wbsp = new WordBreakSpellChecker(); From 8d1e624a6799ed49efa3de489ecf845fc0563cce Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Thu, 11 Jul 2024 11:44:49 -0400 Subject: [PATCH 02/66] Add HnswGraphBuilder.getCompletedGraph() and record completed state (#13561) --- .../lucene99/Lucene99HnswVectorsWriter.java | 2 +- .../apache/lucene/util/hnsw/HnswBuilder.java | 8 ++++++++ .../util/hnsw/HnswConcurrentMergeBuilder.java | 13 ++++++++++++- .../lucene/util/hnsw/HnswGraphBuilder.java | 18 +++++++++++++++++- .../lucene/util/hnsw/HnswGraphTestCase.java | 5 ++++- 5 files changed, 42 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java index bf97426738b..ce7d88f9dfd 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java @@ -615,7 +615,7 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { OnHeapHnswGraph getGraph() { assert flatFieldVectorsWriter.isFinished(); if (node > 0) { - return hnswGraphBuilder.getGraph(); + return hnswGraphBuilder.getCompletedGraph(); } else { return null; } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java index 547385607af..672bf5c6010 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java @@ -41,4 +41,12 @@ public interface HnswBuilder { void setInfoStream(InfoStream infoStream); OnHeapHnswGraph getGraph(); + + /** + * Once this method is called no further updates to the graph are accepted (addGraphNode will + * throw IllegalStateException). Final modifications to the graph (eg patching up disconnected + * components, re-ordering node ids for better delta compression) may be triggered, so callers + * should expect this call to take some time. + */ + OnHeapHnswGraph getCompletedGraph(); } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java index db48960df86..2da7f8492ba 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java @@ -41,6 +41,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { private final TaskExecutor taskExecutor; private final ConcurrentMergeWorker[] workers; private InfoStream infoStream = InfoStream.getDefault(); + private boolean frozen; public HnswConcurrentMergeBuilder( TaskExecutor taskExecutor, @@ -69,6 +70,9 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { @Override public OnHeapHnswGraph build(int maxOrd) throws IOException { + if (frozen) { + throw new IllegalStateException("graph has already been built"); + } if (infoStream.isEnabled(HNSW_COMPONENT)) { infoStream.message( HNSW_COMPONENT, @@ -84,7 +88,8 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { }); } taskExecutor.invokeAll(futures); - return workers[0].getGraph(); + frozen = true; + return workers[0].getCompletedGraph(); } @Override @@ -100,6 +105,12 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { } } + @Override + public OnHeapHnswGraph getCompletedGraph() { + frozen = true; + return getGraph(); + } + @Override public OnHeapHnswGraph getGraph() { return workers[0].getGraph(); diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index b794244105e..2677c327642 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -65,6 +65,7 @@ public class HnswGraphBuilder implements HnswBuilder { protected final OnHeapHnswGraph hnsw; private InfoStream infoStream = InfoStream.getDefault(); + private boolean frozen; public static HnswGraphBuilder create( RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth, long seed) @@ -152,11 +153,14 @@ public class HnswGraphBuilder implements HnswBuilder { @Override public OnHeapHnswGraph build(int maxOrd) throws IOException { + if (frozen) { + throw new IllegalStateException("This HnswGraphBuilder is frozen and cannot be updated"); + } if (infoStream.isEnabled(HNSW_COMPONENT)) { infoStream.message(HNSW_COMPONENT, "build graph from " + maxOrd + " vectors"); } addVectors(maxOrd); - return hnsw; + return getCompletedGraph(); } @Override @@ -164,6 +168,12 @@ public class HnswGraphBuilder implements HnswBuilder { this.infoStream = infoStream; } + @Override + public OnHeapHnswGraph getCompletedGraph() { + frozen = true; + return getGraph(); + } + @Override public OnHeapHnswGraph getGraph() { return hnsw; @@ -171,6 +181,9 @@ public class HnswGraphBuilder implements HnswBuilder { /** add vectors in range [minOrd, maxOrd) */ protected void addVectors(int minOrd, int maxOrd) throws IOException { + if (frozen) { + throw new IllegalStateException("This HnswGraphBuilder is frozen and cannot be updated"); + } long start = System.nanoTime(), t = start; if (infoStream.isEnabled(HNSW_COMPONENT)) { infoStream.message(HNSW_COMPONENT, "addVectors [" + minOrd + " " + maxOrd + ")"); @@ -207,6 +220,9 @@ public class HnswGraphBuilder implements HnswBuilder { to the newly introduced levels (repeating step 2,3 for new levels) and again try to promote the node to entry node. */ + if (frozen) { + throw new IllegalStateException("Graph builder is already frozen"); + } RandomVectorScorer scorer = scorerSupplier.scorer(node); final int nodeLevel = getRandomGraphLevel(ml, random); // first add nodes to all levels diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java index 72a4841581d..09d7721fec9 100644 --- a/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java @@ -218,6 +218,7 @@ abstract class HnswGraphTestCase extends LuceneTestCase { RandomVectorScorerSupplier scorerSupplier = buildScorerSupplier(vectors); HnswGraphBuilder builder = HnswGraphBuilder.create(scorerSupplier, M, beamWidth, seed); HnswGraph hnsw = builder.build(vectors.size()); + expectThrows(IllegalStateException.class, () -> builder.addGraphNode(0)); // Recreate the graph while indexing with the same random seed and write it out HnswGraphBuilder.randSeed = seed; @@ -1014,13 +1015,15 @@ abstract class HnswGraphTestCase extends LuceneTestCase { builder.setBatchSize(100); builder.build(size); exec.shutdownNow(); - OnHeapHnswGraph graph = builder.getGraph(); + OnHeapHnswGraph graph = builder.getCompletedGraph(); assertTrue(graph.entryNode() != -1); assertEquals(size, graph.size()); assertEquals(size - 1, graph.maxNodeId()); for (int l = 0; l < graph.numLevels(); l++) { assertNotNull(graph.getNodesOnLevel(l)); } + // cannot build twice + expectThrows(IllegalStateException.class, () -> builder.build(size)); } public void testAllNodesVisitedInSingleLevel() throws IOException { From cc145553957ec6659bdcaf5ea8e41da840910963 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 12 Jul 2024 09:49:32 +0100 Subject: [PATCH 03/66] Lucene99HnswVectorsReader[.readFields] readability tweaks (#13532) * remove unnecessary readFields parameter * consistently use this. in constructor * align declare and init order --- .../codecs/lucene99/Lucene99HnswVectorsReader.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java index 899140af93a..35bc38571a6 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java @@ -68,10 +68,10 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class); + private final FlatVectorsReader flatVectorsReader; private final FieldInfos fieldInfos; private final Map fields = new HashMap<>(); private final IndexInput vectorIndex; - private final FlatVectorsReader flatVectorsReader; public Lucene99HnswVectorsReader(SegmentReadState state, FlatVectorsReader flatVectorsReader) throws IOException { @@ -93,13 +93,13 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader Lucene99HnswVectorsFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - readFields(meta, state.fieldInfos); + readFields(meta); } catch (Throwable exception) { priorE = exception; } finally { CodecUtil.checkFooter(meta, priorE); } - vectorIndex = + this.vectorIndex = openDataInput( state, versionMeta, @@ -154,9 +154,9 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader } } - private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException { + private void readFields(ChecksumIndexInput meta) throws IOException { for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) { - FieldInfo info = infos.fieldInfo(fieldNumber); + FieldInfo info = fieldInfos.fieldInfo(fieldNumber); if (info == null) { throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); } From c55d664b3e5596acb4137a63d42e9cda9cc6607f Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 12 Jul 2024 10:48:10 +0100 Subject: [PATCH 04/66] in KnnVectorsWriter reduce code duplication w.r.t. MergedVectorValues.merge(Float|Byte)VectorValues (#13539) Co-authored-by: Vigya Sharma --- .../lucene/codecs/KnnVectorsWriter.java | 90 ++++++++++++------- 1 file changed, 56 insertions(+), 34 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java index 053ab893df1..28f9995b11e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.function.BiFunction; import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.DocIDMerger; import org.apache.lucene.index.DocsWithFieldSet; @@ -35,6 +36,7 @@ import org.apache.lucene.internal.hppc.IntIntHashMap; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.VectorScorer; import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.IOFunction; /** Writes vectors to an index. */ public abstract class KnnVectorsWriter implements Accountable, Closeable { @@ -111,11 +113,11 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable { } /** Tracks state of one sub-reader that we are merging */ - private static class VectorValuesSub extends DocIDMerger.Sub { + private static class FloatVectorValuesSub extends DocIDMerger.Sub { final FloatVectorValues values; - VectorValuesSub(MergeState.DocMap docMap, FloatVectorValues values) { + FloatVectorValuesSub(MergeState.DocMap docMap, FloatVectorValues values) { super(docMap); this.values = values; assert values.docID() == -1; @@ -201,61 +203,81 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable { public static final class MergedVectorValues { private MergedVectorValues() {} - /** Returns a merged view over all the segment's {@link FloatVectorValues}. */ - public static FloatVectorValues mergeFloatVectorValues( - FieldInfo fieldInfo, MergeState mergeState) throws IOException { + private static void validateFieldEncoding(FieldInfo fieldInfo, VectorEncoding expected) { assert fieldInfo != null && fieldInfo.hasVectorValues(); - if (fieldInfo.getVectorEncoding() != VectorEncoding.FLOAT32) { + VectorEncoding fieldEncoding = fieldInfo.getVectorEncoding(); + if (fieldEncoding != expected) { throw new UnsupportedOperationException( - "Cannot merge vectors encoded as [" + fieldInfo.getVectorEncoding() + "] as FLOAT32"); + "Cannot merge vectors encoded as [" + fieldEncoding + "] as " + expected); } - List subs = new ArrayList<>(); - for (int i = 0; i < mergeState.knnVectorsReaders.length; i++) { - KnnVectorsReader knnVectorsReader = mergeState.knnVectorsReaders[i]; + } + + private static List mergeVectorValues( + KnnVectorsReader[] knnVectorsReaders, + MergeState.DocMap[] docMaps, + IOFunction valuesSupplier, + BiFunction newSub) + throws IOException { + List subs = new ArrayList<>(); + for (int i = 0; i < knnVectorsReaders.length; i++) { + KnnVectorsReader knnVectorsReader = knnVectorsReaders[i]; if (knnVectorsReader != null) { - FloatVectorValues values = knnVectorsReader.getFloatVectorValues(fieldInfo.name); + V values = valuesSupplier.apply(knnVectorsReader); if (values != null) { - subs.add(new VectorValuesSub(mergeState.docMaps[i], values)); + subs.add(newSub.apply(docMaps[i], values)); } } } - return new MergedFloat32VectorValues(subs, mergeState); + return subs; + } + + /** Returns a merged view over all the segment's {@link FloatVectorValues}. */ + public static FloatVectorValues mergeFloatVectorValues( + FieldInfo fieldInfo, MergeState mergeState) throws IOException { + validateFieldEncoding(fieldInfo, VectorEncoding.FLOAT32); + return new MergedFloat32VectorValues( + mergeVectorValues( + mergeState.knnVectorsReaders, + mergeState.docMaps, + knnVectorsReader -> { + return knnVectorsReader.getFloatVectorValues(fieldInfo.name); + }, + (docMap, values) -> { + return new FloatVectorValuesSub(docMap, values); + }), + mergeState); } /** Returns a merged view over all the segment's {@link ByteVectorValues}. */ public static ByteVectorValues mergeByteVectorValues(FieldInfo fieldInfo, MergeState mergeState) throws IOException { - assert fieldInfo != null && fieldInfo.hasVectorValues(); - if (fieldInfo.getVectorEncoding() != VectorEncoding.BYTE) { - throw new UnsupportedOperationException( - "Cannot merge vectors encoded as [" + fieldInfo.getVectorEncoding() + "] as BYTE"); - } - List subs = new ArrayList<>(); - for (int i = 0; i < mergeState.knnVectorsReaders.length; i++) { - KnnVectorsReader knnVectorsReader = mergeState.knnVectorsReaders[i]; - if (knnVectorsReader != null) { - ByteVectorValues values = knnVectorsReader.getByteVectorValues(fieldInfo.name); - if (values != null) { - subs.add(new ByteVectorValuesSub(mergeState.docMaps[i], values)); - } - } - } - return new MergedByteVectorValues(subs, mergeState); + validateFieldEncoding(fieldInfo, VectorEncoding.BYTE); + return new MergedByteVectorValues( + mergeVectorValues( + mergeState.knnVectorsReaders, + mergeState.docMaps, + knnVectorsReader -> { + return knnVectorsReader.getByteVectorValues(fieldInfo.name); + }, + (docMap, values) -> { + return new ByteVectorValuesSub(docMap, values); + }), + mergeState); } static class MergedFloat32VectorValues extends FloatVectorValues { - private final List subs; - private final DocIDMerger docIdMerger; + private final List subs; + private final DocIDMerger docIdMerger; private final int size; private int docId; - VectorValuesSub current; + FloatVectorValuesSub current; - private MergedFloat32VectorValues(List subs, MergeState mergeState) + private MergedFloat32VectorValues(List subs, MergeState mergeState) throws IOException { this.subs = subs; docIdMerger = DocIDMerger.of(subs, mergeState.needsIndexSort); int totalSize = 0; - for (VectorValuesSub sub : subs) { + for (FloatVectorValuesSub sub : subs) { totalSize += sub.values.size(); } size = totalSize; From 5e52b8094aca4dab2898c7dd17084fc7cf7fd016 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Sun, 14 Jul 2024 09:48:16 -0400 Subject: [PATCH 05/66] Add IntervalsSource for range and regexp queries (#13562) We already have convenient functions for contructing IntervalsSource for wildcard and fuzzy functions. This adds functions for regexp and range as well. --- lucene/CHANGES.txt | 3 + .../lucene/queries/intervals/Intervals.java | 86 +++++++++++++++++++ .../queries/intervals/TestIntervals.java | 69 +++++++++++++++ 3 files changed, 158 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d67318da678..14a6d0eb773 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -261,6 +261,9 @@ Improvements * GITHUB#13548: Refactor and javadoc update for KNN vector writer classes. (Patrick Zhai) +* GITHUB#13562: Add Intervals.regexp and Intervals.range methods to produce IntervalsSource + for regexp and range queries. (Mayya Sharipova) + Optimizations --------------------- diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java index a16e721b5c8..7b9c933c167 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java @@ -27,11 +27,15 @@ import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.index.Term; import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.PrefixQuery; +import org.apache.lucene.search.RegexpQuery; +import org.apache.lucene.search.TermRangeQuery; import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.LevenshteinAutomata; import org.apache.lucene.util.automaton.Operations; +import org.apache.lucene.util.automaton.RegExp; /** * Factory functions for creating {@link IntervalsSource interval sources}. @@ -206,6 +210,88 @@ public final class Intervals { return new MultiTermIntervalsSource(ca, maxExpansions, wildcard.utf8ToString()); } + /** + * Return an {@link IntervalsSource} over the disjunction of all terms that match a regular + * expression + * + * @param regexp regular expression + * @throws IllegalStateException if the regex expands to more than {@link #DEFAULT_MAX_EXPANSIONS} + * terms + * @see RegexpQuery for regexp format + */ + public static IntervalsSource regexp(BytesRef regexp) { + return regexp(regexp, DEFAULT_MAX_EXPANSIONS); + } + + /** + * Expert: Return an {@link IntervalsSource} over the disjunction of all terms that match a + * regular expression + * + *

WARNING: Setting {@code maxExpansions} to higher than the default value of {@link + * #DEFAULT_MAX_EXPANSIONS} can be both slow and memory-intensive + * + * @param regexp regular expression + * @param maxExpansions the maximum number of terms to expand to + * @throws IllegalStateException if the regex expands to more than {@link #DEFAULT_MAX_EXPANSIONS} + * terms + * @see RegexpQuery for regexp format + */ + public static IntervalsSource regexp(BytesRef regexp, int maxExpansions) { + Automaton automaton = new RegExp(new Term("", regexp).text()).toAutomaton(); + CompiledAutomaton ca = new CompiledAutomaton(automaton, false, true, false); + return new MultiTermIntervalsSource(ca, maxExpansions, regexp.utf8ToString()); + } + + /** + * Return an {@link IntervalsSource} over the disjunction of all terms that fall within the given + * range + * + * @param lowerTerm The term text at the lower end of the range + * @param upperTerm The term text at the upper end of the range + * @param includeLower If true, the lowerTerm is included in the range + * @param includeUpper If true, the upperTerm is included in the range + * @throws IllegalStateException if the range expands to more than {@link #DEFAULT_MAX_EXPANSIONS} + * terms + */ + public static IntervalsSource range( + BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) { + return range(lowerTerm, upperTerm, includeLower, includeUpper, DEFAULT_MAX_EXPANSIONS); + } + + /** + * Expert: Return an {@link IntervalsSource} over the disjunction of all terms that fall within + * the given range + * + *

WARNING: Setting {@code maxExpansions} to higher than the default value of {@link + * #DEFAULT_MAX_EXPANSIONS} can be both slow and memory-intensive + * + * @param lowerTerm The term text at the lower end of the range + * @param upperTerm The term text at the upper end of the range + * @param includeLower If true, the lowerTerm is included in the range + * @param includeUpper If true, the upperTerm is included in the range + * @param maxExpansions the maximum number of terms to expand to + * @throws IllegalStateException if the wildcard glob expands to more than {@code maxExpansions} + * terms + */ + public static IntervalsSource range( + BytesRef lowerTerm, + BytesRef upperTerm, + boolean includeLower, + boolean includeUpper, + int maxExpansions) { + Automaton automaton = + TermRangeQuery.toAutomaton(lowerTerm, upperTerm, includeLower, includeUpper); + CompiledAutomaton ca = new CompiledAutomaton(automaton, false, true, true); + + StringBuilder buffer = new StringBuilder(); + buffer.append("{"); + buffer.append(lowerTerm.utf8ToString()); + buffer.append(","); + buffer.append(upperTerm.utf8ToString()); + buffer.append("}"); + return new MultiTermIntervalsSource(ca, maxExpansions, buffer.toString()); + } + /** * A fuzzy term {@link IntervalsSource} matches the disjunction of intervals of terms that are * within the specified {@code maxEdits} from the provided term. diff --git a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java index 6d1413efabd..944530937ce 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java @@ -1028,6 +1028,40 @@ public class TestIntervals extends LuceneTestCase { checkVisits(Intervals.wildcard(new BytesRef("p??")), 1); } + public void testRegexp() throws IOException { + IntervalsSource source = Intervals.regexp(new BytesRef(".ot")); + checkIntervals( + source, + "field1", + 4, + new int[][] { + {}, + {2, 2, 10, 10, 17, 17, 27, 27}, + {5, 5, 10, 10, 21, 21}, + {3, 3}, + {2, 2, 10, 10, 17, 17}, + {} + }); + MatchesIterator mi = getMatches(source, 4, "field1"); + assertNotNull(mi); + assertMatch(mi, 2, 2, 15, 18); + assertMatch(mi, 10, 10, 63, 66); + assertMatch(mi, 17, 17, 97, 100); + + IllegalStateException e = + expectThrows( + IllegalStateException.class, + () -> { + IntervalsSource s = Intervals.regexp(new BytesRef(".ot"), 1); + for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) { + s.intervals("field1", ctx); + } + }); + assertEquals("Automaton [.ot] expanded to too many terms (limit 1)", e.getMessage()); + + checkVisits(Intervals.regexp(new BytesRef("p.*")), 1); + } + public void testFuzzyTerm() throws IOException { IntervalsSource source = Intervals.fuzzyTerm("kot", 1); // matches 'pot' checkIntervals( @@ -1069,6 +1103,41 @@ public class TestIntervals extends LuceneTestCase { checkVisits(Intervals.fuzzyTerm("kot", FuzzyQuery.defaultMaxEdits), 1); } + public void testRange() throws IOException { + IntervalsSource source = Intervals.range(new BytesRef("cold"), new BytesRef("hot"), true, true); + checkIntervals( + source, + "field1", + 6, + new int[][] { + {5, 5}, + {2, 2, 5, 5, 12, 12, 17, 17, 21, 21, 29, 29}, + {2, 2, 5, 5, 12, 12, 17, 17, 21, 21, 27, 27}, + {1, 1, 3, 3, 4, 4}, + {2, 2, 5, 5, 17, 17}, + {2, 2} + }); + MatchesIterator mi = getMatches(source, 3, "field1"); + assertNotNull(mi); + assertMatch(mi, 1, 1, 4, 8); + assertMatch(mi, 3, 3, 15, 18); + assertMatch(mi, 4, 4, 19, 24); + + IllegalStateException e = + expectThrows( + IllegalStateException.class, + () -> { + IntervalsSource s = + Intervals.range(new BytesRef("cold"), new BytesRef("hot"), true, true, 1); + for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) { + s.intervals("field1", ctx); + } + }); + assertEquals("Automaton [{cold,hot}] expanded to too many terms (limit 1)", e.getMessage()); + + checkVisits(source, 1); + } + public void testWrappedFilters() throws IOException { IntervalsSource source = Intervals.or( From 99488b224558e65597a5bd13e9a4ac9e6b012e83 Mon Sep 17 00:00:00 2001 From: Chris Hegarty <62058229+ChrisHegarty@users.noreply.github.com> Date: Wed, 17 Jul 2024 08:56:11 +0100 Subject: [PATCH 06/66] Ensure to use IOContext.READONCE when reading segment files (#13574) This commit uses IOContext.READONCE in more places where the index input is clearly being read once by the thread opening it. We can then enforce that segment files are only opened with READONCE, in the test specific Mock directory wrapper. Much of the changes in this PR update individual test usage, but there is one non-test change to Directory::copyFrom. --- .../core/src/java/org/apache/lucene/store/Directory.java | 4 ++-- .../apache/lucene/index/TestAllFilesCheckIndexHeader.java | 2 +- .../index/TestAllFilesDetectMismatchedChecksum.java | 2 +- .../apache/lucene/index/TestAllFilesDetectTruncation.java | 2 +- .../lucene/index/TestAllFilesHaveChecksumFooter.java | 3 ++- .../apache/lucene/index/TestAllFilesHaveCodecHeader.java | 3 ++- .../apache/lucene/index/TestIndexWriterExceptions.java | 4 ++-- .../test/org/apache/lucene/index/TestSegmentInfos.java | 4 ++-- .../apache/lucene/index/TestSnapshotDeletionPolicy.java | 3 ++- .../org/apache/lucene/luke/models/util/IndexUtils.java | 2 +- .../apache/lucene/tests/store/MockDirectoryWrapper.java | 8 ++++++++ .../java/org/apache/lucene/tests/util/LuceneTestCase.java | 2 +- 12 files changed, 25 insertions(+), 14 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/store/Directory.java b/lucene/core/src/java/org/apache/lucene/store/Directory.java index f25113c4a79..369e6f45473 100644 --- a/lucene/core/src/java/org/apache/lucene/store/Directory.java +++ b/lucene/core/src/java/org/apache/lucene/store/Directory.java @@ -172,12 +172,12 @@ public abstract class Directory implements Closeable { /** * Copies an existing {@code src} file from directory {@code from} to a non-existent file {@code - * dest} in this directory. + * dest} in this directory. The given IOContext is only used for opening the destination file. */ public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException { boolean success = false; - try (IndexInput is = from.openInput(src, context); + try (IndexInput is = from.openInput(src, IOContext.READONCE); IndexOutput os = createOutput(dest, context)) { os.copyBytes(is, is.length()); success = true; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java index 0fe1465fd0c..f826ee45b66 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java @@ -115,7 +115,7 @@ public class TestAllFilesCheckIndexHeader extends LuceneTestCase { // time this will only require one iteration! while (true) { try (IndexOutput out = dirCopy.createOutput(name, IOContext.DEFAULT); - IndexInput in = dir.openInput(name, IOContext.DEFAULT)) { + IndexInput in = dir.openInput(name, IOContext.READONCE)) { // keeps same file length, but replaces the first wrongBytes with random bytes: byte[] bytes = new byte[wrongBytes]; random().nextBytes(bytes); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectMismatchedChecksum.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectMismatchedChecksum.java index ef0fca81aeb..0022a5e3cae 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectMismatchedChecksum.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectMismatchedChecksum.java @@ -122,7 +122,7 @@ public class TestAllFilesDetectMismatchedChecksum extends LuceneTestCase { dirCopy.copyFrom(dir, name, name, IOContext.DEFAULT); } else { try (IndexOutput out = dirCopy.createOutput(name, IOContext.DEFAULT); - IndexInput in = dir.openInput(name, IOContext.DEFAULT)) { + IndexInput in = dir.openInput(name, IOContext.READONCE)) { out.copyBytes(in, flipOffset); out.writeByte((byte) (in.readByte() + TestUtil.nextInt(random(), 0x01, 0xFF))); out.copyBytes(in, victimLength - flipOffset - 1); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java index 6b4abc87a09..b72fa0f2980 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java @@ -152,7 +152,7 @@ public class TestAllFilesDetectTruncation extends LuceneTestCase { } try (IndexOutput out = dirCopy.createOutput(name, IOContext.DEFAULT); - IndexInput in = dir.openInput(name, IOContext.DEFAULT)) { + IndexInput in = dir.openInput(name, IOContext.READONCE)) { out.copyBytes(in, victimLength - lostBytes); } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java index b9dd3656da4..3826962779a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java @@ -19,6 +19,7 @@ package org.apache.lucene.index; import java.io.IOException; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.index.RandomIndexWriter; @@ -77,7 +78,7 @@ public class TestAllFilesHaveChecksumFooter extends LuceneTestCase { } private void checkFooter(Directory dir, String file) throws IOException { - try (IndexInput in = dir.openInput(file, newIOContext(random()))) { + try (IndexInput in = dir.openInput(file, IOContext.READONCE)) { CodecUtil.checksumEntireFile(in); } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java index cced891b86e..76c3ee75f25 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.index.RandomIndexWriter; @@ -84,7 +85,7 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase { private void checkHeader( Directory dir, String file, Map namesToExtensions, byte[] id) throws IOException { - try (IndexInput in = dir.openInput(file, newIOContext(random()))) { + try (IndexInput in = dir.openInput(file, IOContext.READONCE)) { int val = CodecUtil.readBEInt(in); assertEquals( file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index 674ba6eac65..0e7f7814f92 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -1275,7 +1275,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase { assertTrue("segment generation should be > 0 but got " + gen, gen > 0); final String segmentsFileName = SegmentInfos.getLastCommitSegmentsFileName(dir); - IndexInput in = dir.openInput(segmentsFileName, newIOContext(random())); + IndexInput in = dir.openInput(segmentsFileName, IOContext.READONCE); IndexOutput out = dir.createOutput( IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", 1 + gen), @@ -1320,7 +1320,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase { String fileNameIn = SegmentInfos.getLastCommitSegmentsFileName(dir); String fileNameOut = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", 1 + gen); - IndexInput in = dir.openInput(fileNameIn, newIOContext(random())); + IndexInput in = dir.openInput(fileNameIn, IOContext.READONCE); IndexOutput out = dir.createOutput(fileNameOut, newIOContext(random())); long length = in.length(); for (int i = 0; i < length - 1; i++) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java index a4521649b89..90b0a07aa34 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java @@ -367,7 +367,7 @@ public class TestSegmentInfos extends LuceneTestCase { boolean corrupt = false; for (String file : dir.listAll()) { if (file.startsWith(IndexFileNames.SEGMENTS)) { - try (IndexInput in = dir.openInput(file, IOContext.DEFAULT); + try (IndexInput in = dir.openInput(file, IOContext.READONCE); IndexOutput out = corruptDir.createOutput(file, IOContext.DEFAULT)) { final long corruptIndex = TestUtil.nextLong(random(), 0, in.length() - 1); out.copyBytes(in, corruptIndex); @@ -375,7 +375,7 @@ public class TestSegmentInfos extends LuceneTestCase { out.writeByte((byte) b); out.copyBytes(in, in.length() - in.getFilePointer()); } - try (IndexInput in = corruptDir.openInput(file, IOContext.DEFAULT)) { + try (IndexInput in = corruptDir.openInput(file, IOContext.READONCE)) { CodecUtil.checksumEntireFile(in); if (VERBOSE) { System.out.println("TEST: Altering the file did not update the checksum, aborting..."); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java index a341a10349b..0a35e17084c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java @@ -26,6 +26,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.FieldType; import org.apache.lucene.document.TextField; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.util.LuceneTestCase; @@ -230,7 +231,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase { @SuppressForbidden(reason = "Thread sleep") private void readFile(Directory dir, String name) throws Exception { - IndexInput input = dir.openInput(name, newIOContext(random())); + IndexInput input = dir.openInput(name, IOContext.READONCE); try { long size = dir.fileLength(name); long bytesLeft = size; diff --git a/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java b/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java index 735fc6db818..9357a467487 100644 --- a/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java +++ b/lucene/luke/src/java/org/apache/lucene/luke/models/util/IndexUtils.java @@ -341,7 +341,7 @@ public final class IndexUtils { @Override protected String doBody(String segmentFileName) throws IOException { String format = "unknown"; - try (IndexInput in = dir.openInput(segmentFileName, IOContext.DEFAULT)) { + try (IndexInput in = dir.openInput(segmentFileName, IOContext.READONCE)) { if (CodecUtil.CODEC_MAGIC == CodecUtil.readBEInt(in)) { int actualVersion = CodecUtil.checkHeaderNoMagic( diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java index 2589d082fc9..2f30a8cda50 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java @@ -814,6 +814,14 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { context = LuceneTestCase.newIOContext(randomState, context); final boolean confined = context == IOContext.READONCE; + if (name.startsWith(IndexFileNames.SEGMENTS) && confined == false) { + throw new RuntimeException( + "MockDirectoryWrapper: opening segments file [" + + name + + "] with a non-READONCE context[" + + context + + "]"); + } IndexInput delegateInput = in.openInput(name, context); final IndexInput ii; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java index c61968d557e..91942f660f9 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java @@ -2989,7 +2989,7 @@ public abstract class LuceneTestCase extends Assert { */ public static boolean slowFileExists(Directory dir, String fileName) throws IOException { try { - dir.openInput(fileName, IOContext.DEFAULT).close(); + dir.openInput(fileName, IOContext.READONCE).close(); return true; } catch (@SuppressWarnings("unused") NoSuchFileException | FileNotFoundException e) { return false; From 22ca695ef5cef722b9f962c89c6da52c9b3883b0 Mon Sep 17 00:00:00 2001 From: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> Date: Wed, 17 Jul 2024 13:54:19 +0200 Subject: [PATCH 07/66] Add target search concurrency to TieredMergePolicy (#13430) --- lucene/CHANGES.txt | 6 +- .../lucene/index/TieredMergePolicy.java | 80 +++++++++++++++---- .../lucene/index/TestTieredMergePolicy.java | 39 +++++++-- .../lucene/tests/util/LuceneTestCase.java | 6 ++ 4 files changed, 110 insertions(+), 21 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 14a6d0eb773..f361fdcf002 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -254,7 +254,11 @@ API Changes New Features --------------------- -(No changes) + +* GITHUB#13430: Allow configuring the search concurrency via + TieredMergePolicy#setTargetSearchConcurrency. This in-turn instructs the + merge policy to try to have at least this number of segments on the highest + tier. (Adrien Grand, Carlos Delgado) Improvements --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java index 208aa297287..702df660c44 100644 --- a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java @@ -93,6 +93,7 @@ public class TieredMergePolicy extends MergePolicy { private double segsPerTier = 10.0; private double forceMergeDeletesPctAllowed = 10.0; private double deletesPctAllowed = 20.0; + private int targetSearchConcurrency = 1; /** Sole constructor, setting all settings to their defaults. */ public TieredMergePolicy() { @@ -257,6 +258,26 @@ public class TieredMergePolicy extends MergePolicy { return segsPerTier; } + /** + * Sets the target search concurrency. This prevents creating segments that are bigger than + * maxDoc/targetSearchConcurrency, which in turn makes the work parallelizable into + * targetSearchConcurrency slices of similar doc counts. It also makes merging less aggressive, as + * higher values result in indices that do less merging and have more segments + */ + public TieredMergePolicy setTargetSearchConcurrency(int targetSearchConcurrency) { + if (targetSearchConcurrency < 1) { + throw new IllegalArgumentException( + "targetSearchConcurrency must be >= 1 (got " + targetSearchConcurrency + ")"); + } + this.targetSearchConcurrency = targetSearchConcurrency; + return this; + } + + /** Returns the target search concurrency. */ + public int getTargetSearchConcurrency() { + return targetSearchConcurrency; + } + private static class SegmentSizeAndDocs { private final SegmentCommitInfo segInfo; /// Size of the segment in bytes, pro-rated by the number of live documents. @@ -371,31 +392,40 @@ public class TieredMergePolicy extends MergePolicy { // If we have too-large segments, grace them out of the maximum segment count // If we're above certain thresholds of deleted docs, we can merge very large segments. int tooBigCount = 0; + // We relax merging for the bigger segments for concurrency reasons, as we want to have several + // segments on the highest tier without over-merging on the lower tiers. + int concurrencyCount = 0; iter = sortedInfos.iterator(); + double allowedSegCount = 0; + // remove large segments from consideration under two conditions. // 1> Overall percent deleted docs relatively small and this segment is larger than 50% // maxSegSize // 2> overall percent deleted docs large and this segment is large and has few deleted docs - while (iter.hasNext()) { SegmentSizeAndDocs segSizeDocs = iter.next(); double segDelPct = 100 * (double) segSizeDocs.delCount / (double) segSizeDocs.maxDoc; if (segSizeDocs.sizeInBytes > maxMergedSegmentBytes / 2 && (totalDelPct <= deletesPctAllowed || segDelPct <= deletesPctAllowed)) { iter.remove(); - tooBigCount++; // Just for reporting purposes. + tooBigCount++; totIndexBytes -= segSizeDocs.sizeInBytes; allowedDelCount -= segSizeDocs.delCount; + } else if (concurrencyCount + tooBigCount < targetSearchConcurrency - 1) { + // Make sure we count a whole segment for the first targetSearchConcurrency-1 segments to + // avoid over merging on the lower levels. + concurrencyCount++; + allowedSegCount++; + totIndexBytes -= segSizeDocs.sizeInBytes; } } allowedDelCount = Math.max(0, allowedDelCount); final int mergeFactor = (int) Math.min(maxMergeAtOnce, segsPerTier); - // Compute max allowed segments in the index + // Compute max allowed segments for the remainder of the index long levelSize = Math.max(minSegmentBytes, floorSegmentBytes); long bytesLeft = totIndexBytes; - double allowedSegCount = 0; while (true) { final double segCountLevel = bytesLeft / (double) levelSize; if (segCountLevel < segsPerTier || levelSize == maxMergedSegmentBytes) { @@ -408,7 +438,8 @@ public class TieredMergePolicy extends MergePolicy { } // allowedSegCount may occasionally be less than segsPerTier // if segment sizes are below the floor size - allowedSegCount = Math.max(allowedSegCount, segsPerTier); + allowedSegCount = Math.max(allowedSegCount, Math.max(segsPerTier, targetSearchConcurrency)); + int allowedDocCount = getMaxAllowedDocs(totalMaxDoc, totalDelDocs); if (verbose(mergeContext) && tooBigCount > 0) { message( @@ -419,7 +450,11 @@ public class TieredMergePolicy extends MergePolicy { + " (eligible count=" + sortedInfos.size() + ") tooBigCount= " - + tooBigCount, + + tooBigCount + + " allowedDocCount=" + + allowedDocCount + + " vs doc count=" + + infos.totalMaxDoc(), mergeContext); } return doFindMerges( @@ -428,6 +463,7 @@ public class TieredMergePolicy extends MergePolicy { mergeFactor, (int) allowedSegCount, allowedDelCount, + allowedDocCount, MERGE_TYPE.NATURAL, mergeContext, mergingBytes >= maxMergedSegmentBytes); @@ -439,6 +475,7 @@ public class TieredMergePolicy extends MergePolicy { final int mergeFactor, final int allowedSegCount, final int allowedDelCount, + final int allowedDocCount, final MERGE_TYPE mergeType, MergeContext mergeContext, boolean maxMergeIsRunning) @@ -522,16 +559,23 @@ public class TieredMergePolicy extends MergePolicy { final List candidate = new ArrayList<>(); boolean hitTooLarge = false; long bytesThisMerge = 0; + long docCountThisMerge = 0; for (int idx = startIdx; idx < sortedEligible.size() && candidate.size() < mergeFactor - && bytesThisMerge < maxMergedSegmentBytes; + && bytesThisMerge < maxMergedSegmentBytes + && (bytesThisMerge < floorSegmentBytes || docCountThisMerge <= allowedDocCount); idx++) { final SegmentSizeAndDocs segSizeDocs = sortedEligible.get(idx); final long segBytes = segSizeDocs.sizeInBytes; - - if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes) { - hitTooLarge = true; + int segDocCount = segSizeDocs.maxDoc - segSizeDocs.delCount; + if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes + || (totAfterMergeBytes > floorSegmentBytes + && docCountThisMerge + segDocCount > allowedDocCount)) { + // Only set hitTooLarge when reaching the maximum byte size, as this will create + // segments of the maximum size which will no longer be eligible for merging for a long + // time (until they accumulate enough deletes). + hitTooLarge |= totAfterMergeBytes + segBytes > maxMergedSegmentBytes; if (candidate.size() == 0) { // We should never have something coming in that _cannot_ be merged, so handle // singleton merges @@ -548,6 +592,7 @@ public class TieredMergePolicy extends MergePolicy { } candidate.add(segSizeDocs.segInfo); bytesThisMerge += segBytes; + docCountThisMerge += segDocCount; totAfterMergeBytes += segBytes; } @@ -916,14 +961,13 @@ public class TieredMergePolicy extends MergePolicy { final Set merging = mergeContext.getMergingSegments(); boolean haveWork = false; + int totalDelCount = 0; for (SegmentCommitInfo info : infos) { int delCount = mergeContext.numDeletesToMerge(info); assert assertDelCount(delCount, info); + totalDelCount += delCount; double pctDeletes = 100. * ((double) delCount) / info.info.maxDoc(); - if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) { - haveWork = true; - break; - } + haveWork = haveWork || (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)); } if (haveWork == false) { @@ -950,11 +994,16 @@ public class TieredMergePolicy extends MergePolicy { Integer.MAX_VALUE, Integer.MAX_VALUE, 0, + getMaxAllowedDocs(infos.totalMaxDoc(), totalDelCount), MERGE_TYPE.FORCE_MERGE_DELETES, mergeContext, false); } + int getMaxAllowedDocs(int totalMaxDoc, int totalDelDocs) { + return Math.ceilDiv(totalMaxDoc - totalDelDocs, targetSearchConcurrency); + } + private long floorSize(long bytes) { return Math.max(floorSegmentBytes, bytes); } @@ -969,7 +1018,8 @@ public class TieredMergePolicy extends MergePolicy { sb.append("segmentsPerTier=").append(segsPerTier).append(", "); sb.append("maxCFSSegmentSizeMB=").append(getMaxCFSSegmentSizeMB()).append(", "); sb.append("noCFSRatio=").append(noCFSRatio).append(", "); - sb.append("deletesPctAllowed=").append(deletesPctAllowed); + sb.append("deletesPctAllowed=").append(deletesPctAllowed).append(", "); + sb.append("targetSearchConcurrency=").append(targetSearchConcurrency); return sb.toString(); } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java index 5db9b13eb95..eb24d964702 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java @@ -65,6 +65,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { segmentSizes.add(weightedByteSize); minSegmentBytes = Math.min(minSegmentBytes, weightedByteSize); } + Collections.sort(segmentSizes); final double delPercentage = 100.0 * totalDelCount / totalMaxDoc; assertTrue( @@ -77,12 +78,26 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { long levelSizeBytes = Math.max(minSegmentBytes, (long) (tmp.getFloorSegmentMB() * 1024 * 1024)); long bytesLeft = totalBytes; double allowedSegCount = 0; + List biggestSegments = segmentSizes; + if (biggestSegments.size() > tmp.getTargetSearchConcurrency() - 1) { + biggestSegments = + biggestSegments.subList( + biggestSegments.size() - tmp.getTargetSearchConcurrency() + 1, + biggestSegments.size()); + } + // Allow whole segments for the targetSearchConcurrency-1 biggest segments + for (long size : biggestSegments) { + bytesLeft -= size; + allowedSegCount++; + } + // below we make the assumption that segments that reached the max segment // size divided by 2 don't need merging anymore int mergeFactor = (int) Math.min(tmp.getSegmentsPerTier(), tmp.getMaxMergeAtOnce()); while (true) { final double segCountLevel = bytesLeft / (double) levelSizeBytes; - if (segCountLevel < tmp.getSegmentsPerTier() || levelSizeBytes >= maxMergedSegmentBytes / 2) { + if (segCountLevel <= tmp.getSegmentsPerTier() + || levelSizeBytes >= maxMergedSegmentBytes / 2) { allowedSegCount += Math.ceil(segCountLevel); break; } @@ -94,7 +109,6 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { // It's ok to be over the allowed segment count if none of the most balanced merges are balanced // enough - Collections.sort(segmentSizes); boolean hasBalancedMerges = false; for (int i = 0; i < segmentSizes.size() - mergeFactor; ++i) { long maxMergeSegmentSize = segmentSizes.get(i + mergeFactor - 1); @@ -111,11 +125,24 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { } } + // There can be more segments if we can't merge docs because they are balanced between segments. + // At least the + // 2 smallest segments should be mergeable. + // should be 2 segments to merge + int maxDocsPerSegment = tmp.getMaxAllowedDocs(infos.totalMaxDoc(), totalDelCount); + List segmentDocs = + infos.asList().stream() + .map(info -> info.info.maxDoc() - info.getDelCount()) + .sorted() + .toList(); + boolean eligibleDocsMerge = + segmentDocs.size() >= 2 && segmentDocs.get(0) + segmentDocs.get(1) < maxDocsPerSegment; + int numSegments = infos.asList().size(); assertTrue( String.format( Locale.ROOT, - "mergeFactor=%d minSegmentBytes=%,d maxMergedSegmentBytes=%,d segmentsPerTier=%g maxMergeAtOnce=%d numSegments=%d allowed=%g totalBytes=%,d delPercentage=%g deletesPctAllowed=%g", + "mergeFactor=%d minSegmentBytes=%,d maxMergedSegmentBytes=%,d segmentsPerTier=%g maxMergeAtOnce=%d numSegments=%d allowed=%g totalBytes=%,d delPercentage=%g deletesPctAllowed=%g targetNumSegments=%d", mergeFactor, minSegmentBytes, maxMergedSegmentBytes, @@ -125,8 +152,9 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { allowedSegCount, totalBytes, delPercentage, - tmp.getDeletesPctAllowed()), - numSegments <= allowedSegCount || hasBalancedMerges == false); + tmp.getDeletesPctAllowed(), + tmp.getTargetSearchConcurrency()), + numSegments <= allowedSegCount || hasBalancedMerges == false || eligibleDocsMerge == false); } @Override @@ -208,6 +236,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase { int segmentCount = w.getSegmentCount(); int targetCount = TestUtil.nextInt(random(), 1, segmentCount); + if (VERBOSE) { System.out.println( "TEST: merge to " + targetCount + " segs (current count=" + segmentCount + ")"); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java index 91942f660f9..0f9cab1eda4 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java @@ -1095,6 +1095,12 @@ public abstract class LuceneTestCase extends Assert { } else { tmp.setSegmentsPerTier(TestUtil.nextInt(r, 10, 50)); } + if (rarely(r)) { + tmp.setTargetSearchConcurrency(TestUtil.nextInt(r, 10, 50)); + } else { + tmp.setTargetSearchConcurrency(TestUtil.nextInt(r, 2, 20)); + } + configureRandom(r, tmp); tmp.setDeletesPctAllowed(20 + random().nextDouble() * 30); return tmp; From fff997f80165b41e65b10e1d1eb338133d600052 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Jul 2024 11:23:19 +0200 Subject: [PATCH 08/66] Stop requiring MaxScoreBulkScorer's outer window from having at least INNER_WINDOW_SIZE docs. (#13582) Currently `MaxScoreBulkScorer` requires its "outer" window to be at least `WINDOW_SIZE`. The intuition there was that we should make sure we should use the whole range of the bit set that we are using to collect matches. The downside is that it may force us to use an upper level in the skip list that has worse upper bounds for the scores. --- lucene/CHANGES.txt | 3 +++ .../java/org/apache/lucene/search/MaxScoreBulkScorer.java | 5 ----- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f361fdcf002..bc20db67121 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -292,6 +292,9 @@ Optimizations * GITHUB#12100: WordBreakSpellChecker.suggestWordBreaks now does a breadth first search, allowing it to return better matches with fewer evaluations (hossman) +* GITHUB#13582: Stop requiring MaxScoreBulkScorer's outer window from having at + least INNER_WINDOW_SIZE docs. (Adrien Grand) + Changes in runtime behavior --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index f250abfbcb9..bce02cbdae6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -271,11 +271,6 @@ final class MaxScoreBulkScorer extends BulkScorer { windowMax = (int) Math.min(windowMax, upTo + 1L); // upTo is inclusive } - // Score at least an entire inner window of docs - windowMax = - Math.max( - windowMax, (int) Math.min(Integer.MAX_VALUE, (long) windowMin + INNER_WINDOW_SIZE)); - return windowMax; } From 9f040864a62120cba8f5109cf595e5a74f16e4da Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Jul 2024 11:28:35 +0200 Subject: [PATCH 09/66] Add a `targetSearchConcurrency` parameter to `LogMergePolicy`. (#13517) This adds the same `targetSearchConcurrency` parameter to `LogMergePolicy` that #13430 is adding to `TieredMergePolicy`. --- lucene/CHANGES.txt | 4 ++ .../apache/lucene/index/LogMergePolicy.java | 37 ++++++++++++++++++- .../lucene/TestMergeSchedulerExternal.java | 2 +- .../index/TestConcurrentMergeScheduler.java | 1 + .../index/TestIndexWriterExceptions.java | 1 + .../index/TestIndexWriterMergePolicy.java | 4 +- .../lucene/index/TestIndexWriterReader.java | 8 +++- .../lucene/index/TestLogMergePolicy.java | 6 ++- .../lucene/tests/util/LuceneTestCase.java | 5 ++- 9 files changed, 59 insertions(+), 9 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index bc20db67121..8532ce81f8e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -260,6 +260,10 @@ New Features merge policy to try to have at least this number of segments on the highest tier. (Adrien Grand, Carlos Delgado) +* GITHUB#13517: Allow configuring the search concurrency on LogDocMergePolicy + and LogByteSizeMergePolicy via a new #setTargetConcurrency setter. + (Adrien Grand) + Improvements --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java index f7c5011d9c6..f2113d0d13b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java @@ -90,6 +90,12 @@ public abstract class LogMergePolicy extends MergePolicy { /** If true, we pro-rate a segment's size by the percentage of non-deleted documents. */ protected boolean calibrateSizeByDeletes = true; + /** + * Target search concurrency. This merge policy will avoid creating segments that have more than + * {@code maxDoc / targetSearchConcurrency} documents. + */ + protected int targetSearchConcurrency = 1; + /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */ public LogMergePolicy() { super(DEFAULT_NO_CFS_RATIO, MergePolicy.DEFAULT_MAX_CFS_SEGMENT_SIZE); @@ -131,6 +137,28 @@ public abstract class LogMergePolicy extends MergePolicy { return calibrateSizeByDeletes; } + /** + * Sets the target search concurrency. This prevents creating segments that are bigger than + * maxDoc/targetSearchConcurrency, which in turn makes the work parallelizable into + * targetSearchConcurrency slices of similar doc counts. + * + *

NOTE: Configuring a value greater than 1 will increase the number of segments in the + * index linearly with the value of {@code targetSearchConcurrency} and also increase write + * amplification. + */ + public void setTargetSearchConcurrency(int targetSearchConcurrency) { + if (targetSearchConcurrency < 1) { + throw new IllegalArgumentException( + "targetSearchConcurrency must be >= 1 (got " + targetSearchConcurrency + ")"); + } + this.targetSearchConcurrency = targetSearchConcurrency; + } + + /** Returns the target search concurrency. */ + public int getTargetSearchConcurrency() { + return targetSearchConcurrency; + } + /** * Return the number of documents in the provided {@link SegmentCommitInfo}, pro-rated by * percentage of non-deleted documents if {@link #setCalibrateSizeByDeletes} is set. @@ -484,8 +512,10 @@ public abstract class LogMergePolicy extends MergePolicy { final Set mergingSegments = mergeContext.getMergingSegments(); + int totalDocCount = 0; for (int i = 0; i < numSegments; i++) { final SegmentCommitInfo info = infos.info(i); + totalDocCount += sizeDocs(info, mergeContext); long size = size(info, mergeContext); // Floor tiny segments @@ -575,6 +605,9 @@ public abstract class LogMergePolicy extends MergePolicy { mergeContext); } + final int maxMergeDocs = + Math.min(this.maxMergeDocs, Math.ceilDiv(totalDocCount, targetSearchConcurrency)); + // Finally, record all merges that are viable at this level: int end = start + mergeFactor; while (end <= 1 + upto) { @@ -590,7 +623,9 @@ public abstract class LogMergePolicy extends MergePolicy { } long segmentSize = size(info, mergeContext); long segmentDocs = sizeDocs(info, mergeContext); - if (mergeSize + segmentSize > maxMergeSize || mergeDocs + segmentDocs > maxMergeDocs) { + if (mergeSize + segmentSize > maxMergeSize + || (mergeSize + segmentSize > minMergeSize + && mergeDocs + segmentDocs > maxMergeDocs)) { // This merge is full, stop adding more segments to it if (i == start) { // This segment alone is too large, return a singleton merge diff --git a/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java b/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java index 506a76f9989..99e1e8e1fc0 100644 --- a/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java +++ b/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java @@ -132,7 +132,7 @@ public class TestMergeSchedulerExternal extends LuceneTestCase { logMP.setMergeFactor(10); try { - for (int i = 0; i < 20; i++) { + for (int i = 0; i < 60; i++) { writer.addDocument(doc); } } catch ( diff --git a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java index 5f85b5d3774..e0b2c49d854 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java @@ -802,6 +802,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase { iwc.setMaxBufferedDocs(2); LogMergePolicy lmp = newLogMergePolicy(); lmp.setMergeFactor(2); + lmp.setTargetSearchConcurrency(1); iwc.setMergePolicy(lmp); IndexWriter w = new IndexWriter(dir, iwc); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index 0e7f7814f92..5757141b8f2 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -471,6 +471,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase { cms.setSuppressExceptions(); conf.setMergeScheduler(cms); ((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(2); + ((LogMergePolicy) conf.getMergePolicy()).setTargetSearchConcurrency(1); TestPoint3 testPoint = new TestPoint3(); IndexWriter w = RandomIndexWriter.mockIndexWriter(random(), dir, conf, testPoint); testPoint.doFail = true; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java index 1835a4ff309..c56feb62d84 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java @@ -900,8 +900,10 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase { } public void testSetDiagnostics() throws IOException { + LogMergePolicy logMp = newLogMergePolicy(4); + logMp.setTargetSearchConcurrency(1); MergePolicy myMergePolicy = - new FilterMergePolicy(newLogMergePolicy(4)) { + new FilterMergePolicy(logMp) { @Override public MergeSpecification findMerges( MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java index 51b502a122b..82aa548df80 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java @@ -999,6 +999,8 @@ public class TestIndexWriterReader extends LuceneTestCase { public void testSegmentWarmer() throws Exception { Directory dir = newDirectory(); final AtomicBoolean didWarm = new AtomicBoolean(); + LogMergePolicy mp = newLogMergePolicy(10); + mp.setTargetSearchConcurrency(1); IndexWriter w = new IndexWriter( dir, @@ -1012,7 +1014,7 @@ public class TestIndexWriterReader extends LuceneTestCase { assertEquals(20, count); didWarm.set(true); }) - .setMergePolicy(newLogMergePolicy(10))); + .setMergePolicy(mp)); Document doc = new Document(); doc.add(newStringField("foo", "bar", Field.Store.NO)); @@ -1045,6 +1047,8 @@ public class TestIndexWriterReader extends LuceneTestCase { return true; } }; + LogMergePolicy mp = newLogMergePolicy(10); + mp.setTargetSearchConcurrency(1); IndexWriter w = new IndexWriter( dir, @@ -1053,7 +1057,7 @@ public class TestIndexWriterReader extends LuceneTestCase { .setReaderPooling(true) .setInfoStream(infoStream) .setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(infoStream)) - .setMergePolicy(newLogMergePolicy(10))); + .setMergePolicy(mp)); Document doc = new Document(); doc.add(newStringField("foo", "bar", Field.Store.NO)); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java index 0b00a20a17f..ea60f9b1e09 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java @@ -249,8 +249,10 @@ public class TestLogMergePolicy extends BaseMergePolicyTestCase { SegmentInfos segmentInfos = new SegmentInfos(Version.LATEST.major); LogMergePolicy mp = mergePolicy(); + // Number of segments guaranteed to trigger a merge. + int numSegmentsForMerging = mp.getMergeFactor() + mp.getTargetSearchConcurrency(); - for (int i = 0; i < mp.getMergeFactor(); ++i) { + for (int i = 0; i < numSegmentsForMerging; ++i) { segmentInfos.add( makeSegmentCommitInfo( "_" + segNameGenerator.getAndIncrement(), @@ -266,6 +268,6 @@ public class TestLogMergePolicy extends BaseMergePolicyTestCase { segmentInfos = applyMerge(segmentInfos, merge, "_" + segNameGenerator.getAndIncrement(), stats); } - assertEquals(1, segmentInfos.size()); + assertTrue(segmentInfos.size() < numSegmentsForMerging); } } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java index 0f9cab1eda4..3299d8ddca9 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java @@ -1053,6 +1053,7 @@ public abstract class LuceneTestCase extends Assert { public static LogMergePolicy newLogMergePolicy(Random r) { LogMergePolicy logmp = r.nextBoolean() ? new LogDocMergePolicy() : new LogByteSizeMergePolicy(); logmp.setCalibrateSizeByDeletes(r.nextBoolean()); + logmp.setTargetSearchConcurrency(TestUtil.nextInt(random(), 1, 16)); if (rarely(r)) { logmp.setMergeFactor(TestUtil.nextInt(r, 2, 9)); } else { @@ -1112,14 +1113,14 @@ public abstract class LuceneTestCase extends Assert { return logmp; } - public static MergePolicy newLogMergePolicy(boolean useCFS, int mergeFactor) { + public static LogMergePolicy newLogMergePolicy(boolean useCFS, int mergeFactor) { LogMergePolicy logmp = newLogMergePolicy(); logmp.setNoCFSRatio(useCFS ? 1.0 : 0.0); logmp.setMergeFactor(mergeFactor); return logmp; } - public static MergePolicy newLogMergePolicy(int mergeFactor) { + public static LogMergePolicy newLogMergePolicy(int mergeFactor) { LogMergePolicy logmp = newLogMergePolicy(); logmp.setMergeFactor(mergeFactor); return logmp; From 00c9d9a03c90fd85469e2889e755c0e51e27b015 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Jul 2024 11:52:16 +0200 Subject: [PATCH 10/66] Fix test failures due to #13517. --- .../core/src/java/org/apache/lucene/index/LogMergePolicy.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java index f2113d0d13b..a86809a1fb1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java @@ -623,9 +623,7 @@ public abstract class LogMergePolicy extends MergePolicy { } long segmentSize = size(info, mergeContext); long segmentDocs = sizeDocs(info, mergeContext); - if (mergeSize + segmentSize > maxMergeSize - || (mergeSize + segmentSize > minMergeSize - && mergeDocs + segmentDocs > maxMergeDocs)) { + if (mergeSize + segmentSize > maxMergeSize || mergeDocs + segmentDocs > maxMergeDocs) { // This merge is full, stop adding more segments to it if (i == start) { // This segment alone is too large, return a singleton merge From b42fd8e479c1f2aaae4d668905cd0498086d9723 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 18 Jul 2024 22:47:44 -0700 Subject: [PATCH 11/66] Avoid wrap readers without soft-deletes (#13588) I analyzed a heap dump of Elasticsearch where FixedBitSet uses more than 1GB of memory. Most of these FixedBitSets are used by soft-deletes reader wrappers, even though these segments have no deletes at all. I believe these segments previously had soft-deletes, but these deletes were pruned by merges. The reason we wrap soft-deletes is that the soft-deletes field exists. Since these segments had soft-deletes previously, we carried the field-infos into the new segment. Ideally, we should have ways to check whether the returned docValues iterator is empty or not so that we can avoid allocating FixedBitSet completely, or we should prune fields without values after merges. --- .../SoftDeletesDirectoryReaderWrapper.java | 3 ++ ...TestSoftDeletesDirectoryReaderWrapper.java | 52 +++++++++++++++++++ 2 files changed, 55 insertions(+) diff --git a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesDirectoryReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesDirectoryReaderWrapper.java index 09048742aa8..1515c8469c1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesDirectoryReaderWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesDirectoryReaderWrapper.java @@ -141,6 +141,9 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead bits.set(0, reader.maxDoc()); } int numSoftDeletes = PendingSoftDeletes.applySoftDeletes(iterator, bits); + if (numSoftDeletes == 0) { + return reader; + } int numDeletes = reader.numDeletedDocs() + numSoftDeletes; int numDocs = reader.maxDoc() - numDeletes; assert assertDocCounts(numDocs, numSoftDeletes, reader); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java index d1adaeb0e9b..2098f57910d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java @@ -17,6 +17,8 @@ package org.apache.lucene.index; +import static org.hamcrest.Matchers.instanceOf; + import java.io.IOException; import java.util.HashSet; import java.util.Set; @@ -27,6 +29,7 @@ import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StringField; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.util.LuceneTestCase; @@ -264,4 +267,53 @@ public class TestSoftDeletesDirectoryReaderWrapper extends LuceneTestCase { assertEquals(1, leafCalled.get()); IOUtils.close(reader, writer, dir); } + + public void testAvoidWrappingReadersWithoutSoftDeletes() throws Exception { + IndexWriterConfig iwc = newIndexWriterConfig(); + String softDeletesField = "soft_deletes"; + iwc.setSoftDeletesField(softDeletesField); + MergePolicy mergePolicy = iwc.mergePolicy; + iwc.setMergePolicy( + new SoftDeletesRetentionMergePolicy(softDeletesField, MatchAllDocsQuery::new, mergePolicy)); + try (Directory dir = newDirectory(); + IndexWriter writer = new IndexWriter(dir, iwc)) { + int numDocs = 1 + random().nextInt(10); + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + String docId = Integer.toString(i); + doc.add(new StringField("id", docId, Field.Store.YES)); + writer.addDocument(doc); + } + int numDeletes = 1 + random().nextInt(5); + for (int i = 0; i < numDeletes; i++) { + Document doc = new Document(); + String docId = Integer.toString(random().nextInt(numDocs)); + doc.add(new StringField("id", docId, Field.Store.YES)); + writer.softUpdateDocument( + new Term("id", docId), doc, new NumericDocValuesField(softDeletesField, 0)); + } + writer.flush(); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + SoftDeletesDirectoryReaderWrapper wrapped = + new SoftDeletesDirectoryReaderWrapper(reader, softDeletesField); + assertEquals(numDocs, wrapped.numDocs()); + assertEquals(numDeletes, wrapped.numDeletedDocs()); + } + writer + .getConfig() + .setMergePolicy( + new SoftDeletesRetentionMergePolicy( + softDeletesField, MatchNoDocsQuery::new, mergePolicy)); + writer.forceMerge(1); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + SoftDeletesDirectoryReaderWrapper wrapped = + new SoftDeletesDirectoryReaderWrapper(reader, softDeletesField); + assertEquals(numDocs, wrapped.numDocs()); + assertEquals(0, wrapped.numDeletedDocs()); + for (LeafReaderContext leaf : wrapped.leaves()) { + assertThat(leaf.reader(), instanceOf(SegmentReader.class)); + } + } + } + } } From c245ed2fb4502c096554b08ed68837d9b320bfb5 Mon Sep 17 00:00:00 2001 From: zhouhui Date: Fri, 19 Jul 2024 16:12:52 +0800 Subject: [PATCH 12/66] Remove useless todo. (#13589) --- .../java/org/apache/lucene/tests/index/PerThreadPKLookup.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/PerThreadPKLookup.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/PerThreadPKLookup.java index c1823e8b708..0a259b7f179 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/PerThreadPKLookup.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/PerThreadPKLookup.java @@ -85,8 +85,6 @@ public class PerThreadPKLookup { if (termsEnums[seg].seekExact(id)) { postingsEnums[seg] = termsEnums[seg].postings(postingsEnums[seg], 0); int docID = -1; - // TODO: Can we get postings' last Doc directly? and return the last one we find. - // TODO: Maybe we should check liveDoc whether null out of the loop? while ((docID = postingsEnums[seg].nextDoc()) != PostingsEnum.NO_MORE_DOCS) { if (liveDocs[seg] == null || liveDocs[seg].get(docID)) { return docBases[seg] + docID; From 9f991ed07e15c2b357690fe5ab50ba30f48bb4d2 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Fri, 19 Jul 2024 11:20:16 +0200 Subject: [PATCH 13/66] Add levels to DocValues skipper index (#13563) Adding levels o be able to skip several intervals in one step. --- lucene/CHANGES.txt | 2 + .../lucene90/Lucene90DocValuesConsumer.java | 110 ++++++++++++++---- .../lucene90/Lucene90DocValuesFormat.java | 31 +++++ .../lucene90/Lucene90DocValuesProducer.java | 69 +++++++---- .../org/apache/lucene/index/CheckIndex.java | 18 +-- ...90DocValuesFormatVariableSkipInterval.java | 3 +- .../index/BaseDocValuesFormatTestCase.java | 21 +++- 7 files changed, 198 insertions(+), 56 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 8532ce81f8e..f1c687c8969 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -133,6 +133,8 @@ New Features DocValuesSkipper abstraction. A new flag is added to FieldType.java that configures whether to create a "skip index" for doc values. (Ignacio Vera) +* GITHUB#13563: Add levels to doc values skip index. (Ignacio Vera) + Improvements --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java index 021eacacd3b..3a8f0a2c8a3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java @@ -19,9 +19,13 @@ package org.apache.lucene.codecs.lucene90; import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT; import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_LEVEL_SHIFT; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_MAX_LEVEL; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; @@ -43,7 +47,6 @@ import org.apache.lucene.search.SortedSetSelector; import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; -import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; @@ -207,65 +210,130 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer { maxValue = Math.max(maxValue, value); } + void accumulate(SkipAccumulator other) { + assert minDocID <= other.minDocID && maxDocID < other.maxDocID; + maxDocID = other.maxDocID; + minValue = Math.min(minValue, other.minValue); + maxValue = Math.max(maxValue, other.maxValue); + docCount += other.docCount; + } + void nextDoc(int docID) { maxDocID = docID; ++docCount; } - void writeTo(DataOutput output) throws IOException { - output.writeInt(maxDocID); - output.writeInt(minDocID); - output.writeLong(maxValue); - output.writeLong(minValue); - output.writeInt(docCount); + public static SkipAccumulator merge(List list, int index, int length) { + SkipAccumulator acc = new SkipAccumulator(list.get(index).minDocID); + for (int i = 0; i < length; i++) { + acc.accumulate(list.get(index + i)); + } + return acc; } } private void writeSkipIndex(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { assert field.hasDocValuesSkipIndex(); - // TODO: This disk compression once we introduce levels - long start = data.getFilePointer(); - SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + final long start = data.getFilePointer(); + final SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); long globalMaxValue = Long.MIN_VALUE; long globalMinValue = Long.MAX_VALUE; int globalDocCount = 0; int maxDocId = -1; + final List accumulators = new ArrayList<>(); SkipAccumulator accumulator = null; - int counter = 0; + final int maxAccumulators = 1 << (SKIP_INDEX_LEVEL_SHIFT * (SKIP_INDEX_MAX_LEVEL - 1)); for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { - if (counter == 0) { + if (accumulator == null) { accumulator = new SkipAccumulator(doc); + accumulators.add(accumulator); } accumulator.nextDoc(doc); for (int i = 0, end = values.docValueCount(); i < end; ++i) { accumulator.accumulate(values.nextValue()); } - if (++counter == skipIndexIntervalSize) { + if (accumulator.docCount == skipIndexIntervalSize) { globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); globalMinValue = Math.min(globalMinValue, accumulator.minValue); globalDocCount += accumulator.docCount; maxDocId = accumulator.maxDocID; - accumulator.writeTo(data); - counter = 0; + accumulator = null; + if (accumulators.size() == maxAccumulators) { + writeLevels(accumulators); + accumulators.clear(); + } } } - if (counter > 0) { - globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); - globalMinValue = Math.min(globalMinValue, accumulator.minValue); - globalDocCount += accumulator.docCount; - maxDocId = accumulator.maxDocID; - accumulator.writeTo(data); + if (accumulators.isEmpty() == false) { + if (accumulator != null) { + globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); + globalMinValue = Math.min(globalMinValue, accumulator.minValue); + globalDocCount += accumulator.docCount; + maxDocId = accumulator.maxDocID; + } + writeLevels(accumulators); } meta.writeLong(start); // record the start in meta meta.writeLong(data.getFilePointer() - start); // record the length + assert globalDocCount == 0 || globalMaxValue >= globalMinValue; meta.writeLong(globalMaxValue); meta.writeLong(globalMinValue); + assert globalDocCount <= maxDocId + 1; meta.writeInt(globalDocCount); meta.writeInt(maxDocId); } + private void writeLevels(List accumulators) throws IOException { + final List> accumulatorsLevels = new ArrayList<>(SKIP_INDEX_MAX_LEVEL); + accumulatorsLevels.add(accumulators); + for (int i = 0; i < SKIP_INDEX_MAX_LEVEL - 1; i++) { + accumulatorsLevels.add(buildLevel(accumulatorsLevels.get(i))); + } + int totalAccumulators = accumulators.size(); + for (int index = 0; index < totalAccumulators; index++) { + // compute how many levels we need to write for the current accumulator + final int levels = getLevels(index, totalAccumulators); + // write the number of levels + data.writeByte((byte) levels); + // write intervals in reverse order. This is done so we don't + // need to read all of them in case of slipping + for (int level = levels - 1; level >= 0; level--) { + final SkipAccumulator accumulator = + accumulatorsLevels.get(level).get(index >> (SKIP_INDEX_LEVEL_SHIFT * level)); + data.writeInt(accumulator.maxDocID); + data.writeInt(accumulator.minDocID); + data.writeLong(accumulator.maxValue); + data.writeLong(accumulator.minValue); + data.writeInt(accumulator.docCount); + } + } + } + + private static List buildLevel(List accumulators) { + final int levelSize = 1 << SKIP_INDEX_LEVEL_SHIFT; + final List collector = new ArrayList<>(); + for (int i = 0; i < accumulators.size() - levelSize + 1; i += levelSize) { + collector.add(SkipAccumulator.merge(accumulators, i, levelSize)); + } + return collector; + } + + private static int getLevels(int index, int size) { + if (Integer.numberOfTrailingZeros(index) >= SKIP_INDEX_LEVEL_SHIFT) { + // TODO: can we do it in constant time rather than linearly with SKIP_INDEX_MAX_LEVEL? + final int left = size - index; + for (int level = SKIP_INDEX_MAX_LEVEL - 1; level > 0; level--) { + final int numberIntervals = 1 << (SKIP_INDEX_LEVEL_SHIFT * level); + if (left >= numberIntervals && index % numberIntervals == 0) { + return level + 1; + } + } + } + return 1; + } + private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer, boolean ords) throws IOException { SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java index 0ae0a7ac2aa..922bc2481c9 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java @@ -194,5 +194,36 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat { static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT; static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1; + // number of documents in an interval private static final int DEFAULT_SKIP_INDEX_INTERVAL_SIZE = 4096; + // bytes on an interval: + // * 1 byte : number of levels + // * 16 bytes: min / max value, + // * 8 bytes: min / max docID + // * 4 bytes: number of documents + private static final long SKIP_INDEX_INTERVAL_BYTES = 29L; + // number of intervals represented as a shift to create a new level, this is 1 << 3 == 8 + // intervals. + static final int SKIP_INDEX_LEVEL_SHIFT = 3; + // max number of levels + // Increasing this number, it increases how much heap we need at index time. + // we currently need (1 * 8 * 8 * 8) = 512 accumulators on heap + static final int SKIP_INDEX_MAX_LEVEL = 4; + // number of bytes to skip when skipping a level. It does not take into account the + // current interval that is being read. + static final long[] SKIP_INDEX_JUMP_LENGTH_PER_LEVEL = new long[SKIP_INDEX_MAX_LEVEL]; + + static { + // Size of the interval minus read bytes (1 byte for level and 4 bytes for maxDocID) + SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[0] = SKIP_INDEX_INTERVAL_BYTES - 5L; + for (int level = 1; level < SKIP_INDEX_MAX_LEVEL; level++) { + // jump from previous level + SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] = SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level - 1]; + // nodes added by new level + SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] += + (1 << (level * SKIP_INDEX_LEVEL_SHIFT)) * SKIP_INDEX_INTERVAL_BYTES; + // remove the byte levels added in the previous level + SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] -= (1 << ((level - 1) * SKIP_INDEX_LEVEL_SHIFT)); + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java index ad467fbc87f..a44092dbc24 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java @@ -16,6 +16,8 @@ */ package org.apache.lucene.codecs.lucene90; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL; +import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_MAX_LEVEL; import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT; import java.io.IOException; @@ -1792,28 +1794,55 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { if (input.length() > 0) { input.prefetch(0, 1); } + // TODO: should we write to disk the actual max level for this segment? return new DocValuesSkipper() { - int minDocID = -1; - int maxDocID = -1; - long minValue, maxValue; - int docCount; + final int[] minDocID = new int[SKIP_INDEX_MAX_LEVEL]; + final int[] maxDocID = new int[SKIP_INDEX_MAX_LEVEL]; + + { + for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) { + minDocID[i] = maxDocID[i] = -1; + } + } + + final long[] minValue = new long[SKIP_INDEX_MAX_LEVEL]; + final long[] maxValue = new long[SKIP_INDEX_MAX_LEVEL]; + final int[] docCount = new int[SKIP_INDEX_MAX_LEVEL]; + int levels = 1; @Override public void advance(int target) throws IOException { if (target > entry.maxDocId) { - minDocID = DocIdSetIterator.NO_MORE_DOCS; - maxDocID = DocIdSetIterator.NO_MORE_DOCS; + // skipper is exhausted + for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) { + minDocID[i] = maxDocID[i] = DocIdSetIterator.NO_MORE_DOCS; + } } else { + // find next interval + assert target > maxDocID[0] : "target must be bigger that current interval"; while (true) { - maxDocID = input.readInt(); - if (maxDocID >= target) { - minDocID = input.readInt(); - maxValue = input.readLong(); - minValue = input.readLong(); - docCount = input.readInt(); + levels = input.readByte(); + assert levels <= SKIP_INDEX_MAX_LEVEL && levels > 0 + : "level out of range [" + levels + "]"; + boolean valid = true; + // check if current interval is competitive or we can jump to the next position + for (int level = levels - 1; level >= 0; level--) { + if ((maxDocID[level] = input.readInt()) < target) { + input.skipBytes(SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level]); // the jump for the level + valid = false; + break; + } + minDocID[level] = input.readInt(); + maxValue[level] = input.readLong(); + minValue[level] = input.readLong(); + docCount[level] = input.readInt(); + } + if (valid) { + // adjust levels + while (levels < SKIP_INDEX_MAX_LEVEL && maxDocID[levels] >= target) { + levels++; + } break; - } else { - input.skipBytes(24); } } } @@ -1821,32 +1850,32 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public int numLevels() { - return 1; + return levels; } @Override public int minDocID(int level) { - return minDocID; + return minDocID[level]; } @Override public int maxDocID(int level) { - return maxDocID; + return maxDocID[level]; } @Override public long minValue(int level) { - return minValue; + return minValue[level]; } @Override public long maxValue(int level) { - return maxValue; + return maxValue[level]; } @Override public int docCount(int level) { - return docCount; + return docCount[level]; } @Override diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index aaa76f418a9..65ac2fcd260 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -3302,17 +3302,17 @@ public final class CheckIndex implements Closeable { if (skipper.maxDocID(0) == NO_MORE_DOCS) { break; } + if (skipper.minDocID(0) < doc) { + throw new CheckIndexException( + "skipper dv iterator for field: " + + fieldName + + " reports wrong minDocID, got " + + skipper.minDocID(0) + + " < " + + doc); + } int levels = skipper.numLevels(); for (int level = 0; level < levels; level++) { - if (skipper.minDocID(level) < doc) { - throw new CheckIndexException( - "skipper dv iterator for field: " - + fieldName - + " reports wrong minDocID, got " - + skipper.minDocID(level) - + " < " - + doc); - } if (skipper.minDocID(level) > skipper.maxDocID(level)) { throw new CheckIndexException( "skipper dv iterator for field: " diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java index 94204fb2a8d..e452313fbd7 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java @@ -25,7 +25,8 @@ public class TestLucene90DocValuesFormatVariableSkipInterval extends BaseDocValu @Override protected Codec getCodec() { - return TestUtil.alwaysDocValuesFormat(new Lucene90DocValuesFormat(random().nextInt(2, 1024))); + // small interval size to test with many intervals + return TestUtil.alwaysDocValuesFormat(new Lucene90DocValuesFormat(random().nextInt(4, 16))); } public void testSkipIndexIntervalSize() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java index 16f33460991..a312b42a910 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java @@ -773,6 +773,13 @@ public abstract class BaseDocValuesFormatTestCase extends LegacyBaseDocValuesFor maxVal <= skipper.maxValue(0)); } docCount += skipper.docCount(0); + for (int level = 1; level < skipper.numLevels(); level++) { + assertTrue(skipper.minDocID(0) >= skipper.minDocID(level)); + assertTrue(skipper.maxDocID(0) <= skipper.maxDocID(level)); + assertTrue(skipper.minValue(0) >= skipper.minValue(level)); + assertTrue(skipper.maxValue(0) <= skipper.maxValue(level)); + assertTrue(skipper.docCount(0) < skipper.docCount(level)); + } } assertEquals(docCount, skipper.docCount()); @@ -784,19 +791,23 @@ public abstract class BaseDocValuesFormatTestCase extends LegacyBaseDocValuesFor if (skipper == null) { return; } + int nextLevel = 0; while (true) { - int doc = random().nextInt(skipper.maxDocID(0), maxDoc + 1) + 1; + int doc = random().nextInt(skipper.maxDocID(nextLevel), maxDoc + 1) + 1; skipper.advance(doc); if (skipper.minDocID(0) == NO_MORE_DOCS) { assertEquals(NO_MORE_DOCS, skipper.maxDocID(0)); return; } if (iterator.advanceExact(doc)) { - assertTrue(iterator.docID() >= skipper.minDocID(0)); - assertTrue(iterator.docID() <= skipper.maxDocID(0)); - assertTrue(iterator.minValue() >= skipper.minValue(0)); - assertTrue(iterator.maxValue() <= skipper.maxValue(0)); + for (int level = 0; level < skipper.numLevels(); level++) { + assertTrue(iterator.docID() >= skipper.minDocID(level)); + assertTrue(iterator.docID() <= skipper.maxDocID(level)); + assertTrue(iterator.minValue() >= skipper.minValue(level)); + assertTrue(iterator.maxValue() <= skipper.maxValue(level)); + } } + nextLevel = random().nextInt(skipper.numLevels()); } } From cc3b412183c7acbb4a6c498e9babffba4e7d1f28 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 19 Jul 2024 15:18:08 +0200 Subject: [PATCH 14/66] Fix test failure --- .../src/test/org/apache/lucene/index/TestIndexSorting.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java index ed776b371c2..0432f05d94f 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java @@ -154,7 +154,9 @@ public class TestIndexSorting extends LuceneTestCase { Sort indexSort = new Sort(sortField, new SortField("id", SortField.Type.INT)); iwc.setIndexSort(indexSort); LogMergePolicy policy = newLogMergePolicy(); - // make sure that merge factor is always > 2 + // make sure that merge factor is always > 2 and target search concurrency is no more than 1 to + // avoid creating merges that are accidentally sorted + policy.setTargetSearchConcurrency(1); if (policy.getMergeFactor() <= 2) { policy.setMergeFactor(3); } From 7709f575efecead357fc29802784c19cc2c1f428 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Mon, 22 Jul 2024 14:52:44 +0200 Subject: [PATCH 15/66] Align doc value skipper interval boundaries when an interval contains a constant value (#13597) keep adding documents to an skipper interval while it is dense and single valued. --- lucene/CHANGES.txt | 3 + .../lucene90/Lucene90DocValuesConsumer.java | 45 +++-- ...90DocValuesFormatVariableSkipInterval.java | 170 +++++++++++++++++- 3 files changed, 202 insertions(+), 16 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f1c687c8969..6de4246129f 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -135,6 +135,9 @@ New Features * GITHUB#13563: Add levels to doc values skip index. (Ignacio Vera) +* GITHUB#13597: Align doc value skipper interval boundaries when an interval contains a constant + value. (Ignacio Vera) + Improvements --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java index 3a8f0a2c8a3..c9f2e774233 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java @@ -205,6 +205,20 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer { docCount = 0; } + boolean isDone(int skipIndexIntervalSize, int valueCount, long nextValue, int nextDoc) { + if (docCount < skipIndexIntervalSize) { + return false; + } + // Once we reach the interval size, we will keep accepting documents if + // - next doc value is not a multi-value + // - current accumulator only contains a single value and next value is the same value + // - the accumulator is dense and the next doc keeps the density (no gaps) + return valueCount > 1 + || minValue != maxValue + || minValue != nextValue + || docCount != nextDoc - minDocID; + } + void accumulate(long value) { minValue = Math.min(minValue, value); maxValue = Math.max(maxValue, value); @@ -245,15 +259,9 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer { SkipAccumulator accumulator = null; final int maxAccumulators = 1 << (SKIP_INDEX_LEVEL_SHIFT * (SKIP_INDEX_MAX_LEVEL - 1)); for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { - if (accumulator == null) { - accumulator = new SkipAccumulator(doc); - accumulators.add(accumulator); - } - accumulator.nextDoc(doc); - for (int i = 0, end = values.docValueCount(); i < end; ++i) { - accumulator.accumulate(values.nextValue()); - } - if (accumulator.docCount == skipIndexIntervalSize) { + final long firstValue = values.nextValue(); + if (accumulator != null + && accumulator.isDone(skipIndexIntervalSize, values.docValueCount(), firstValue, doc)) { globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); globalMinValue = Math.min(globalMinValue, accumulator.minValue); globalDocCount += accumulator.docCount; @@ -264,15 +272,22 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer { accumulators.clear(); } } + if (accumulator == null) { + accumulator = new SkipAccumulator(doc); + accumulators.add(accumulator); + } + accumulator.nextDoc(doc); + accumulator.accumulate(firstValue); + for (int i = 1, end = values.docValueCount(); i < end; ++i) { + accumulator.accumulate(values.nextValue()); + } } if (accumulators.isEmpty() == false) { - if (accumulator != null) { - globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); - globalMinValue = Math.min(globalMinValue, accumulator.minValue); - globalDocCount += accumulator.docCount; - maxDocId = accumulator.maxDocID; - } + globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue); + globalMinValue = Math.min(globalMinValue, accumulator.minValue); + globalDocCount += accumulator.docCount; + maxDocId = accumulator.maxDocID; writeLevels(accumulators); } meta.writeLong(start); // record the start in meta diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java index e452313fbd7..b5fc29440b7 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java @@ -16,11 +16,24 @@ */ package org.apache.lucene.codecs.lucene90; +import java.io.IOException; +import java.util.Arrays; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DocValuesSkipper; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseDocValuesFormatTestCase; +import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.TestUtil; -/** Tests Lucene90DocValuesFormat */ +/** Tests Lucene90DocValuesFormat with custom skipper interval size */ public class TestLucene90DocValuesFormatVariableSkipInterval extends BaseDocValuesFormatTestCase { @Override @@ -36,4 +49,159 @@ public class TestLucene90DocValuesFormatVariableSkipInterval extends BaseDocValu () -> new Lucene90DocValuesFormat(random().nextInt(Integer.MIN_VALUE, 2))); assertTrue(ex.getMessage().contains("skipIndexIntervalSize must be > 1")); } + + public void testSkipperAllEqualValue() throws IOException { + final IndexWriterConfig config = new IndexWriterConfig().setCodec(getCodec()); + try (Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config)) { + final int numDocs = atLeast(100); + for (int i = 0; i < numDocs; i++) { + final Document doc = new Document(); + doc.add(NumericDocValuesField.indexedField("dv", 0L)); + writer.addDocument(doc); + } + writer.forceMerge(1); + try (IndexReader reader = writer.getReader()) { + assertEquals(1, reader.leaves().size()); + final DocValuesSkipper skipper = reader.leaves().get(0).reader().getDocValuesSkipper("dv"); + assertNotNull(skipper); + skipper.advance(0); + assertEquals(0L, skipper.minValue(0)); + assertEquals(0L, skipper.maxValue(0)); + assertEquals(numDocs, skipper.docCount(0)); + skipper.advance(skipper.maxDocID(0) + 1); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, skipper.minDocID(0)); + } + } + } + + // break on different value + public void testSkipperFewValuesSorted() throws IOException { + final IndexWriterConfig config = new IndexWriterConfig().setCodec(getCodec()); + boolean reverse = random().nextBoolean(); + config.setIndexSort(new Sort(new SortField("dv", SortField.Type.LONG, reverse))); + try (Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config)) { + final int intervals = random().nextInt(2, 10); + final int[] numDocs = new int[intervals]; + for (int i = 0; i < intervals; i++) { + numDocs[i] = random().nextInt(10) + 16; + for (int j = 0; j < numDocs[i]; j++) { + final Document doc = new Document(); + doc.add(NumericDocValuesField.indexedField("dv", i)); + writer.addDocument(doc); + } + } + writer.forceMerge(1); + try (IndexReader reader = writer.getReader()) { + assertEquals(1, reader.leaves().size()); + final DocValuesSkipper skipper = reader.leaves().get(0).reader().getDocValuesSkipper("dv"); + assertNotNull(skipper); + assertEquals(Arrays.stream(numDocs).sum(), skipper.docCount()); + skipper.advance(0); + if (reverse) { + for (int i = intervals - 1; i >= 0; i--) { + assertEquals(i, skipper.minValue(0)); + assertEquals(i, skipper.maxValue(0)); + assertEquals(numDocs[i], skipper.docCount(0)); + skipper.advance(skipper.maxDocID(0) + 1); + } + } else { + for (int i = 0; i < intervals; i++) { + assertEquals(i, skipper.minValue(0)); + assertEquals(i, skipper.maxValue(0)); + assertEquals(numDocs[i], skipper.docCount(0)); + skipper.advance(skipper.maxDocID(0) + 1); + } + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, skipper.minDocID(0)); + } + } + } + + // break on empty doc values + public void testSkipperAllEqualValueWithGaps() throws IOException { + final IndexWriterConfig config = new IndexWriterConfig().setCodec(getCodec()); + config.setIndexSort(new Sort(new SortField("sort", SortField.Type.LONG, false))); + try (Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config)) { + final int gaps = random().nextInt(2, 10); + final int[] numDocs = new int[gaps]; + long totaldocs = 0; + for (int i = 0; i < gaps; i++) { + numDocs[i] = random().nextInt(10) + 16; + for (int j = 0; j < numDocs[i]; j++) { + final Document doc = new Document(); + doc.add(new NumericDocValuesField("sort", totaldocs++)); + doc.add(SortedNumericDocValuesField.indexedField("dv", 0L)); + writer.addDocument(doc); + } + // add doc with empty "dv" + final Document doc = new Document(); + doc.add(new NumericDocValuesField("sort", totaldocs++)); + writer.addDocument(doc); + } + writer.forceMerge(1); + try (IndexReader reader = writer.getReader()) { + assertEquals(1, reader.leaves().size()); + final DocValuesSkipper skipper = reader.leaves().get(0).reader().getDocValuesSkipper("dv"); + assertNotNull(skipper); + assertEquals(Arrays.stream(numDocs).sum(), skipper.docCount()); + skipper.advance(0); + for (int i = 0; i < gaps; i++) { + assertEquals(0L, skipper.minValue(0)); + assertEquals(0L, skipper.maxValue(0)); + assertEquals(numDocs[i], skipper.docCount(0)); + skipper.advance(skipper.maxDocID(0) + 1); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, skipper.minDocID(0)); + } + } + } + + // break on multi-values + public void testSkipperAllEqualValueWithMultiValues() throws IOException { + final IndexWriterConfig config = new IndexWriterConfig().setCodec(getCodec()); + config.setIndexSort(new Sort(new SortField("sort", SortField.Type.LONG, false))); + try (Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config)) { + final int gaps = random().nextInt(2, 10); + final int[] numDocs = new int[gaps]; + long totaldocs = 0; + for (int i = 0; i < gaps; i++) { + int docs = random().nextInt(10) + 16; + numDocs[i] += docs; + for (int j = 0; j < docs; j++) { + final Document doc = new Document(); + doc.add(new NumericDocValuesField("sort", totaldocs++)); + doc.add(SortedNumericDocValuesField.indexedField("dv", 0L)); + writer.addDocument(doc); + } + if (i != gaps - 1) { + // add doc with mutivalues + final Document doc = new Document(); + doc.add(new NumericDocValuesField("sort", totaldocs++)); + doc.add(SortedNumericDocValuesField.indexedField("dv", 0L)); + doc.add(SortedNumericDocValuesField.indexedField("dv", 0L)); + writer.addDocument(doc); + numDocs[i + 1] = 1; + } + } + writer.forceMerge(1); + try (IndexReader reader = writer.getReader()) { + assertEquals(1, reader.leaves().size()); + final DocValuesSkipper skipper = reader.leaves().get(0).reader().getDocValuesSkipper("dv"); + assertNotNull(skipper); + assertEquals(Arrays.stream(numDocs).sum(), skipper.docCount()); + skipper.advance(0); + for (int i = 0; i < gaps; i++) { + assertEquals(0L, skipper.minValue(0)); + assertEquals(0L, skipper.maxValue(0)); + assertEquals(numDocs[i], skipper.docCount(0)); + skipper.advance(skipper.maxDocID(0) + 1); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, skipper.minDocID(0)); + } + } + } } From 1c3925cb150d125d987a56ef9603f4490e9ea43f Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Mon, 22 Jul 2024 11:37:01 -0400 Subject: [PATCH 16/66] reconcile main's copy to match 9.10.0 released CHANGES.txt entry --- lucene/CHANGES.txt | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 6de4246129f..a1474f44b4e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -80,10 +80,6 @@ API Changes * GITHUB#12875: Ensure token position is always increased in PathHierarchyTokenizer and ReversePathHierarchyTokenizer and resulting tokens do not overlap. (Michael Froh, Lukáš Vlček) -* GITHUB#12624, GITHUB#12831: Allow FSTCompiler to stream to any DataOutput while building, and - make compile() only return the FSTMetadata. For on-heap (default) use case, please use - FST.fromFSTReader(fstMetadata, fstCompiler.getFSTReader()) to create the FST. (Anh Dung Bui) - * GITHUB#13146, GITHUB#13148: Remove ByteBufferIndexInput and only use MemorySegment APIs for MMapDirectory. (Uwe Schindler) @@ -553,6 +549,10 @@ API Changes * GITHUB#12854: Mark DrillSideways#createDrillDownFacetsCollector as @Deprecated. (Greg Miller) +* GITHUB#12624, GITHUB#12831: Allow FSTCompiler to stream to any DataOutput while building, and + make compile() only return the FSTMetadata. For on-heap (default) use case, please use + FST.fromFSTReader(fstMetadata, fstCompiler.getFSTReader()) to create the FST. (Anh Dung Bui) + New Features --------------------- * GITHUB#12679: Add support for similarity-based vector searches using [Byte|Float]VectorSimilarityQuery. Uses a new @@ -560,6 +560,12 @@ New Features better-scoring nodes are available, or the best candidate is below a score of `traversalSimilarity` in the lowest level. (Aditya Prakash, Kaival Parikh) +* GITHUB#12829: For indices newly created as of 9.10.0 onwards, IndexWriter preserves document blocks indexed via + IndexWriter#addDocuments or IndexWriter#updateDocuments also when index sorting is configured. Document blocks are + maintained alongside their parent documents during sort and merge. IndexWriterConfig accepts a parent field that is used + to maintain block orders if index sorting is used. Note, this is fully optional in Lucene 9.x while will be mandatory for + indices that use document blocks together with index sorting as of 10.0.0. (Simon Willnauer) + * GITHUB#12336: Index additional data per facet label in the taxonomy. (Shai Erera, Egor Potemkin, Mike McCandless, Stefan Vodita) @@ -651,7 +657,6 @@ Build Other --------------------- - * GITHUB#11023: Removing some dead code in CheckIndex. (Jakub Slowinski) * GITHUB#11023: Removing @lucene.experimental tags in testXXX methods in CheckIndex. (Jakub Slowinski) From af9a2b98033918f0796c9838517771302b89ff62 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 22 Jul 2024 11:37:34 -0400 Subject: [PATCH 17/66] Add simple tool to diff entries in lucene's CHANGES.txt that should be identical (#12860) * add simple tool to diff entries in lucene's CHANGES.txt that should be identical * remove temporary debugging code --- dev-tools/scripts/diff_lucene_changes.py | 78 ++++++++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 dev-tools/scripts/diff_lucene_changes.py diff --git a/dev-tools/scripts/diff_lucene_changes.py b/dev-tools/scripts/diff_lucene_changes.py new file mode 100644 index 00000000000..91b7c84994d --- /dev/null +++ b/dev-tools/scripts/diff_lucene_changes.py @@ -0,0 +1,78 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- +# 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. + +import os +import re +import subprocess +import sys +import tempfile +import urllib.request + +''' +A simple tool to see diffs between main's version of CHANGES.txt entries for +a given release vs the stable branch's version. It's best to keep these 1) +identical and 2) matching what changes were actually backported to be honest +to users and avoid future annoying conflicts on backport. +''' + +# e.g. python3 -u diff_lucene_changes.py branch_9_9 main 9.9.0 + +# + +def get_changes_url(branch_name): + if os.path.isdir(branch_name): + url = f'file://{branch_name}/lucene/CHANGES.txt' + else: + url = f'https://raw.githubusercontent.com/apache/lucene/{branch_name}/lucene/CHANGES.txt' + print(f'NOTE: resolving {branch_name} --> {url}') + return url + +def extract_release_section(changes_txt, release_name): + return re.search(f'=======+ Lucene {re.escape(release_name)} =======+(.*?)=======+ Lucene .*? =======+$', + changes_txt.decode('utf-8'), re.MULTILINE | re.DOTALL).group(1).encode('utf-8') + +def main(): + if len(sys.argv) < 3 or len(sys.argv) > 5: + print('\nUsage: python3 -u dev-tools/scripts/diff_lucene_changes.py [diff-commandline-extras]\n') + print(' e.g.: python3 -u dev-tools/scripts/diff_lucene_changes.py branch_9_9 /l/trunk 9.9.0 "-w"\n') + sys.exit(1) + + branch1 = sys.argv[1] + branch2 = sys.argv[2] + release_name = sys.argv[3] + + if len(sys.argv) > 4: + diff_cl_extras = [sys.argv[4]] + else: + diff_cl_extras = [] + + branch1_changes = extract_release_section(urllib.request.urlopen(get_changes_url(branch1)).read(), + release_name) + branch2_changes = extract_release_section(urllib.request.urlopen(get_changes_url(branch2)).read(), + release_name) + + with tempfile.NamedTemporaryFile() as f1, tempfile.NamedTemporaryFile() as f2: + f1.write(branch1_changes) + f2.write(branch2_changes) + + command = ['diff'] + diff_cl_extras + [f1.name, f2.name] + + # diff returns non-zero exit status when there are diffs, so don't pass check=True + print(subprocess.run(command, check=False, capture_output=True).stdout.decode('utf-8')) + +if __name__ == '__main__': + main() From 97d89c661f9e93adb8e0479d8814d8d55f6a5752 Mon Sep 17 00:00:00 2001 From: Dzung Bui Date: Tue, 23 Jul 2024 01:14:53 +0900 Subject: [PATCH 18/66] Refactor FST.saveMetadata() to FSTMetadata.save() (#13549) * lazily write the FST padding byte * Also write the pad byte when there is emptyOutput * add comment * Make Lucene90BlockTreeTermsWriter to write FST off-heap * Add change log * Tidy code & Add comments * use temp IndexOutput for FST writing * Use IOUtils to delete files * Update CHANGES.txt * Update CHANGES.txt --- .../java/org/apache/lucene/util/fst/FST.java | 102 ++++++++++-------- 1 file changed, 55 insertions(+), 47 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java index 17201194da4..ead90dca24c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java +++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java @@ -528,56 +528,10 @@ public final class FST implements Accountable { * @param out the DataOutput to write the FST bytes to */ public void save(DataOutput metaOut, DataOutput out) throws IOException { - saveMetadata(metaOut); + metadata.save(metaOut); fstReader.writeTo(out); } - /** - * Save the metadata to a DataOutput - * - * @param metaOut the DataOutput to write the metadata to - */ - public void saveMetadata(DataOutput metaOut) throws IOException { - CodecUtil.writeHeader(metaOut, FILE_FORMAT_NAME, VERSION_CURRENT); - // TODO: really we should encode this as an arc, arriving - // to the root node, instead of special casing here: - if (metadata.emptyOutput != null) { - // Accepts empty string - metaOut.writeByte((byte) 1); - - // Serialize empty-string output: - ByteBuffersDataOutput ros = new ByteBuffersDataOutput(); - outputs.writeFinalOutput(metadata.emptyOutput, ros); - byte[] emptyOutputBytes = ros.toArrayCopy(); - int emptyLen = emptyOutputBytes.length; - - // reverse - final int stopAt = emptyLen / 2; - int upto = 0; - while (upto < stopAt) { - final byte b = emptyOutputBytes[upto]; - emptyOutputBytes[upto] = emptyOutputBytes[emptyLen - upto - 1]; - emptyOutputBytes[emptyLen - upto - 1] = b; - upto++; - } - metaOut.writeVInt(emptyLen); - metaOut.writeBytes(emptyOutputBytes, 0, emptyLen); - } else { - metaOut.writeByte((byte) 0); - } - final byte t; - if (metadata.inputType == INPUT_TYPE.BYTE1) { - t = 0; - } else if (metadata.inputType == INPUT_TYPE.BYTE2) { - t = 1; - } else { - t = 2; - } - metaOut.writeByte(t); - metaOut.writeVLong(metadata.startNode); - metaOut.writeVLong(numBytes()); - } - /** Writes an automaton to a file. */ public void save(final Path path) throws IOException { try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(path))) { @@ -1249,5 +1203,59 @@ public final class FST implements Accountable { public int getVersion() { return version; } + + public T getEmptyOutput() { + return emptyOutput; + } + + public long getNumBytes() { + return numBytes; + } + + /** + * Save the metadata to a DataOutput + * + * @param metaOut the DataOutput to write the metadata to + */ + public void save(DataOutput metaOut) throws IOException { + CodecUtil.writeHeader(metaOut, FILE_FORMAT_NAME, VERSION_CURRENT); + // TODO: really we should encode this as an arc, arriving + // to the root node, instead of special casing here: + if (emptyOutput != null) { + // Accepts empty string + metaOut.writeByte((byte) 1); + + // Serialize empty-string output: + ByteBuffersDataOutput ros = new ByteBuffersDataOutput(); + outputs.writeFinalOutput(emptyOutput, ros); + byte[] emptyOutputBytes = ros.toArrayCopy(); + int emptyLen = emptyOutputBytes.length; + + // reverse + final int stopAt = emptyLen / 2; + int upto = 0; + while (upto < stopAt) { + final byte b = emptyOutputBytes[upto]; + emptyOutputBytes[upto] = emptyOutputBytes[emptyLen - upto - 1]; + emptyOutputBytes[emptyLen - upto - 1] = b; + upto++; + } + metaOut.writeVInt(emptyLen); + metaOut.writeBytes(emptyOutputBytes, 0, emptyLen); + } else { + metaOut.writeByte((byte) 0); + } + final byte t; + if (inputType == INPUT_TYPE.BYTE1) { + t = 0; + } else if (inputType == INPUT_TYPE.BYTE2) { + t = 1; + } else { + t = 2; + } + metaOut.writeByte(t); + metaOut.writeVLong(startNode); + metaOut.writeVLong(numBytes); + } } } From d491dfe1315c80319547ec183be3d7aa902d2e9e Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 24 Jul 2024 13:06:31 +0200 Subject: [PATCH 19/66] Update TestTopDocsCollector to no longer rely on the deprecated search(Query, Collector) (#13600) --- .../lucene/search/TestTopDocsCollector.java | 38 +++++++++++++++---- 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java index e28be104ac0..0566061f357 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java @@ -18,6 +18,7 @@ package org.apache.lucene.search; import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Store; @@ -41,6 +42,35 @@ import org.apache.lucene.util.BytesRef; public class TestTopDocsCollector extends LuceneTestCase { + private static final class MyTopDocsCollectorMananger + implements CollectorManager { + + private final int numHits; + + MyTopDocsCollectorMananger(int numHits) { + this.numHits = numHits; + } + + @Override + public MyTopDocsCollector newCollector() { + return new MyTopDocsCollector(numHits); + } + + @Override + public MyTopDocsCollector reduce(Collection collectors) { + int totalHits = 0; + MyTopDocsCollector myTopDocsCollector = new MyTopDocsCollector(numHits); + for (MyTopDocsCollector collector : collectors) { + totalHits += collector.totalHits; + for (ScoreDoc scoreDoc : collector.pq) { + myTopDocsCollector.pq.insertWithOverflow(scoreDoc); + } + } + myTopDocsCollector.totalHits = totalHits; + return myTopDocsCollector; + } + } + private static final class MyTopDocsCollector extends TopDocsCollector { private int idx = 0; @@ -125,14 +155,8 @@ public class TestTopDocsCollector extends LuceneTestCase { private TopDocsCollector doSearch(int numResults) throws IOException { Query q = new MatchAllDocsQuery(); - return doSearch(numResults, q); - } - - private TopDocsCollector doSearch(int numResults, Query q) throws IOException { IndexSearcher searcher = newSearcher(reader); - TopDocsCollector tdc = new MyTopDocsCollector(numResults); - searcher.search(q, tdc); - return tdc; + return searcher.search(q, new MyTopDocsCollectorMananger(numResults)); } private TopDocs doSearchWithThreshold( From 97d066dd6b0d354193730d9f3c1f72f046f7dc6a Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 24 Jul 2024 13:07:08 +0200 Subject: [PATCH 20/66] Update TestTopDocsMerge to not rely on search(Query, Collector) (#13601) Relates to #12892 --- .../lucene/search/TestTopDocsMerge.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java index 6bc0931fe63..e683b39c310 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java @@ -292,26 +292,25 @@ public class TestTopDocsMerge extends LuceneTestCase { topHits = searcher.search(query, numHits); } } else { - final TopFieldCollector c = TopFieldCollector.create(sort, numHits, Integer.MAX_VALUE); - searcher.search(query, c); + TopFieldDocs topFieldDocs = + searcher.search(query, new TopFieldCollectorManager(sort, numHits, Integer.MAX_VALUE)); if (useFrom) { from = TestUtil.nextInt(random(), 0, numHits - 1); size = numHits - from; - TopDocs tempTopHits = c.topDocs(); - if (from < tempTopHits.scoreDocs.length) { + if (from < topFieldDocs.scoreDocs.length) { // Can't use TopDocs#topDocs(start, howMany), since it has different behaviour when // start >= hitCount // than TopDocs#merge currently has ScoreDoc[] newScoreDocs = - new ScoreDoc[Math.min(size, tempTopHits.scoreDocs.length - from)]; - System.arraycopy(tempTopHits.scoreDocs, from, newScoreDocs, 0, newScoreDocs.length); - tempTopHits.scoreDocs = newScoreDocs; - topHits = tempTopHits; + new ScoreDoc[Math.min(size, topFieldDocs.scoreDocs.length - from)]; + System.arraycopy(topFieldDocs.scoreDocs, from, newScoreDocs, 0, newScoreDocs.length); + topFieldDocs.scoreDocs = newScoreDocs; + topHits = topFieldDocs; } else { - topHits = new TopDocs(tempTopHits.totalHits, new ScoreDoc[0]); + topHits = new TopDocs(topFieldDocs.totalHits, new ScoreDoc[0]); } } else { - topHits = c.topDocs(0, numHits); + topHits = topFieldDocs; } } From acbd7141404d503e7ca89bec520a175482d41bce Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 24 Jul 2024 14:58:53 +0200 Subject: [PATCH 21/66] Further reduce the search concurrency overhead. (#13606) This iterates on #13546 to further reduce the overhead of search concurrency by caching whether the hit count threshold has been reached: once the threshold has been reached, it cannot get "un-reached" again, so we don't need to pay the cost of `LongAdder#longValue`. --- .../apache/lucene/search/HitsThresholdChecker.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java b/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java index 4232af53fe6..43ff4fecdbb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java +++ b/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java @@ -24,6 +24,10 @@ abstract class HitsThresholdChecker { /** Implementation of HitsThresholdChecker which allows global hit counting */ private static class GlobalHitsThresholdChecker extends HitsThresholdChecker { private final LongAdder globalHitCount = new LongAdder(); + // Cache whether the threshold has been reached already. It is not volatile or synchronized on + // purpose to contain the overhead of reading the value similarly to what String#hashCode() + // does. This does not affect correctness. + private boolean thresholdReached = false; GlobalHitsThresholdChecker(int totalHitsThreshold) { super(totalHitsThreshold); @@ -32,12 +36,17 @@ abstract class HitsThresholdChecker { @Override void incrementHitCount() { - globalHitCount.increment(); + if (thresholdReached == false) { + globalHitCount.increment(); + } } @Override boolean isThresholdReached() { - return globalHitCount.longValue() > getHitsThreshold(); + if (thresholdReached) { + return true; + } + return thresholdReached = globalHitCount.longValue() > getHitsThreshold(); } @Override From 4c1d50d8e8cdd49f4f5011d6647add599baf2dac Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Jul 2024 21:53:30 +0200 Subject: [PATCH 22/66] Save allocating some zero length byte arrays (#13608) Something I found in a heap dump. For large numbers of `FieldReader` where the minimum term is an empty string, we allocate MBs worth of empty `byte[]` in ES. Worth adding the conditional here I think. --- .../lucene90/blocktree/Lucene90BlockTreeTermsReader.java | 3 +-- lucene/core/src/java/org/apache/lucene/util/BytesRef.java | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java index 3871945e0a4..c2ff50215a3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java @@ -270,9 +270,8 @@ public final class Lucene90BlockTreeTermsReader extends FieldsProducer { throw new CorruptIndexException("invalid bytes length: " + numBytes, in); } - BytesRef bytes = new BytesRef(); + BytesRef bytes = new BytesRef(numBytes); bytes.length = numBytes; - bytes.bytes = new byte[numBytes]; in.readBytes(bytes.bytes, 0, numBytes); return bytes; diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java index 351259e87b5..4298668468f 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BytesRef.java +++ b/lucene/core/src/java/org/apache/lucene/util/BytesRef.java @@ -68,7 +68,7 @@ public final class BytesRef implements Comparable, Cloneable { * both be zero. */ public BytesRef(int capacity) { - this.bytes = new byte[capacity]; + this.bytes = capacity == 0 ? EMPTY_BYTES : new byte[capacity]; } /** @@ -77,7 +77,7 @@ public final class BytesRef implements Comparable, Cloneable { * @param text This must be well-formed unicode text, with no unpaired surrogates. */ public BytesRef(CharSequence text) { - this(new byte[UnicodeUtil.maxUTF8Length(text.length())]); + this(UnicodeUtil.maxUTF8Length(text.length())); length = UnicodeUtil.UTF16toUTF8(text, 0, text.length(), bytes); } From b4fb425c43ad37bc8ce754ec4358633538396263 Mon Sep 17 00:00:00 2001 From: Chris Hegarty <62058229+ChrisHegarty@users.noreply.github.com> Date: Thu, 25 Jul 2024 10:12:02 +0100 Subject: [PATCH 23/66] Aggregate files from the same segment into a single Arena (#13570) This commit adds a ref counted shared arena to support aggregating segment files into a single Arena. --- lucene/CHANGES.txt | 7 + .../apache/lucene/store/MMapDirectory.java | 115 ++++++++++++- .../MemorySegmentIndexInputProvider.java | 69 +++++++- .../lucene/store/RefCountedSharedArena.java | 146 ++++++++++++++++ .../lucene/store/TestMMapDirectory.java | 157 ++++++++++++++++++ 5 files changed, 483 insertions(+), 11 deletions(-) create mode 100644 lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index a1474f44b4e..b761a4fa058 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -300,6 +300,13 @@ Optimizations * GITHUB#13582: Stop requiring MaxScoreBulkScorer's outer window from having at least INNER_WINDOW_SIZE docs. (Adrien Grand) +* GITHUB#13570, GITHUB#13574, GITHUB#13535: Avoid performance degradation with closing shared Arenas. + Closing many individual index files can potentially lead to a degradation in execution performance. + Index files are mmapped one-to-one with the JDK's foreign shared Arena. The JVM deoptimizes the top + few frames of all threads when closing a shared Arena (see JDK-8335480). We mitigate this situation + by 1) using a confined Arena where appropriate, and 2) grouping files from the same segment to a + single shared Arena. (Chris Hegarty, Michael Gibney, Uwe Schindler) + Changes in runtime behavior --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java index 7ea1b74b3e8..8b064f4e3b8 100644 --- a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java +++ b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java @@ -16,14 +16,20 @@ */ package org.apache.lucene.store; +import static org.apache.lucene.index.IndexFileNames.CODEC_FILE_PATTERN; + import java.io.IOException; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; import java.nio.channels.ClosedChannelException; // javadoc @link import java.nio.file.Path; import java.util.Locale; +import java.util.Optional; import java.util.concurrent.Future; import java.util.function.BiPredicate; +import java.util.function.Function; +import java.util.logging.Logger; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.util.Constants; /** @@ -42,6 +48,11 @@ import org.apache.lucene.util.Constants; * performance of searches on a cold page cache at the expense of slowing down opening an index. See * {@link #setPreload(BiPredicate)} for more details. * + *

This class supports grouping of files that are part of the same logical group. This is a hint + * that allows for better handling of resources. For example, individual files that are part of the + * same segment can be considered part of the same logical group. See {@link + * #setGroupingFunction(Function)} for more details. + * *

This class will use the modern {@link java.lang.foreign.MemorySegment} API available since * Java 21 which allows to safely unmap previously mmapped files after closing the {@link * IndexInput}s. There is no need to enable the "preview feature" of your Java version; it works out @@ -83,6 +94,41 @@ public class MMapDirectory extends FSDirectory { */ public static final BiPredicate NO_FILES = (filename, context) -> false; + /** + * This sysprop allows to control the total maximum number of mmapped files that can be associated + * with a single shared {@link java.lang.foreign.Arena foreign Arena}. For example, to set the max + * number of permits to 256, pass the following on the command line pass {@code + * -Dorg.apache.lucene.store.MMapDirectory.sharedArenaMaxPermits=256}. Setting a value of 1 + * associates one file to one shared arena. + * + * @lucene.internal + */ + public static final String SHARED_ARENA_MAX_PERMITS_SYSPROP = + "org.apache.lucene.store.MMapDirectory.sharedArenaMaxPermits"; + + /** Argument for {@link #setGroupingFunction(Function)} that configures no grouping. */ + public static final Function> NO_GROUPING = filename -> Optional.empty(); + + /** Argument for {@link #setGroupingFunction(Function)} that configures grouping by segment. */ + public static final Function> GROUP_BY_SEGMENT = + filename -> { + if (!CODEC_FILE_PATTERN.matcher(filename).matches()) { + return Optional.empty(); + } + String groupKey = IndexFileNames.parseSegmentName(filename).substring(1); + try { + // keep the original generation (=0) in base group, later generations in extra group + if (IndexFileNames.parseGeneration(filename) > 0) { + groupKey += "-g"; + } + } catch ( + @SuppressWarnings("unused") + NumberFormatException unused) { + // does not confirm to the generation syntax, or trash + } + return Optional.of(groupKey); + }; + /** * Argument for {@link #setPreload(BiPredicate)} that configures files to be preloaded upon * opening them if they use the {@link ReadAdvice#RANDOM_PRELOAD} advice. @@ -102,6 +148,11 @@ public class MMapDirectory extends FSDirectory { */ public static final long DEFAULT_MAX_CHUNK_SIZE; + /** A provider specific context object or null, that will be passed to openInput. */ + final Object attachment = PROVIDER.attachment(); + + private Function> groupingFunction = GROUP_BY_SEGMENT; + final int chunkSizePower; /** @@ -184,6 +235,21 @@ public class MMapDirectory extends FSDirectory { this.preload = preload; } + /** + * Configures a grouping function for files that are part of the same logical group. The gathering + * of files into a logical group is a hint that allows for better handling of resources. + * + *

By default, grouping is {@link #GROUP_BY_SEGMENT}. To disable, invoke this method with + * {@link #NO_GROUPING}. + * + * @param groupingFunction a function that accepts a file name and returns an optional group key. + * If the optional is present, then its value is the logical group to which the file belongs. + * Otherwise, the file name if not associated with any logical group. + */ + public void setGroupingFunction(Function> groupingFunction) { + this.groupingFunction = groupingFunction; + } + /** * Returns the current mmap chunk size. * @@ -199,20 +265,37 @@ public class MMapDirectory extends FSDirectory { ensureOpen(); ensureCanRead(name); Path path = directory.resolve(name); - return PROVIDER.openInput(path, context, chunkSizePower, preload.test(name, context)); + return PROVIDER.openInput( + path, + context, + chunkSizePower, + preload.test(name, context), + groupingFunction.apply(name), + attachment); } // visible for tests: - static final MMapIndexInputProvider PROVIDER; + static final MMapIndexInputProvider PROVIDER; - interface MMapIndexInputProvider { - IndexInput openInput(Path path, IOContext context, int chunkSizePower, boolean preload) + interface MMapIndexInputProvider { + IndexInput openInput( + Path path, + IOContext context, + int chunkSizePower, + boolean preload, + Optional group, + A attachment) throws IOException; long getDefaultMaxChunkSize(); boolean supportsMadvise(); + /** An optional attachment of the provider, that will be passed to openInput. */ + default A attachment() { + return null; + } + default IOException convertMapFailedIOException( IOException ioe, String resourceDescription, long bufSize) { final String originalMessage; @@ -256,15 +339,33 @@ public class MMapDirectory extends FSDirectory { } } - private static MMapIndexInputProvider lookupProvider() { + private static int getSharedArenaMaxPermitsSysprop() { + int ret = 1024; // default value + try { + String str = System.getProperty(SHARED_ARENA_MAX_PERMITS_SYSPROP); + if (str != null) { + ret = Integer.parseInt(str); + } + } catch (@SuppressWarnings("unused") NumberFormatException | SecurityException ignored) { + Logger.getLogger(MMapDirectory.class.getName()) + .warning( + "Cannot read sysprop " + + SHARED_ARENA_MAX_PERMITS_SYSPROP + + ", so the default value will be used."); + } + return ret; + } + + private static MMapIndexInputProvider lookupProvider() { + final var maxPermits = getSharedArenaMaxPermitsSysprop(); final var lookup = MethodHandles.lookup(); try { final var cls = lookup.findClass("org.apache.lucene.store.MemorySegmentIndexInputProvider"); // we use method handles, so we do not need to deal with setAccessible as we have private // access through the lookup: - final var constr = lookup.findConstructor(cls, MethodType.methodType(void.class)); + final var constr = lookup.findConstructor(cls, MethodType.methodType(void.class, int.class)); try { - return (MMapIndexInputProvider) constr.invoke(); + return (MMapIndexInputProvider) constr.invoke(maxPermits); } catch (RuntimeException | Error e) { throw e; } catch (Throwable th) { diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java index 08f6149746b..1e5a305219b 100644 --- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java +++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java @@ -24,20 +24,32 @@ import java.nio.channels.FileChannel.MapMode; import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.logging.Logger; import org.apache.lucene.util.Constants; import org.apache.lucene.util.Unwrappable; @SuppressWarnings("preview") -final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexInputProvider { +final class MemorySegmentIndexInputProvider + implements MMapDirectory.MMapIndexInputProvider< + ConcurrentHashMap> { private final Optional nativeAccess; + private final int sharedArenaMaxPermits; - MemorySegmentIndexInputProvider() { + MemorySegmentIndexInputProvider(int maxPermits) { this.nativeAccess = NativeAccess.getImplementation(); + this.sharedArenaMaxPermits = checkMaxPermits(maxPermits); } @Override - public IndexInput openInput(Path path, IOContext context, int chunkSizePower, boolean preload) + public IndexInput openInput( + Path path, + IOContext context, + int chunkSizePower, + boolean preload, + Optional group, + ConcurrentHashMap arenas) throws IOException { final String resourceDescription = "MemorySegmentIndexInput(path=\"" + path.toString() + "\")"; @@ -46,7 +58,7 @@ final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexIn boolean success = false; final boolean confined = context == IOContext.READONCE; - final Arena arena = confined ? Arena.ofConfined() : Arena.ofShared(); + final Arena arena = confined ? Arena.ofConfined() : getSharedArena(group, arenas); try (var fc = FileChannel.open(path, StandardOpenOption.READ)) { final long fileSize = fc.size(); final IndexInput in = @@ -125,4 +137,53 @@ final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexIn } return segments; } + + @Override + public ConcurrentHashMap attachment() { + return new ConcurrentHashMap<>(); + } + + private static int checkMaxPermits(int maxPermits) { + if (RefCountedSharedArena.validMaxPermits(maxPermits)) { + return maxPermits; + } + Logger.getLogger(MemorySegmentIndexInputProvider.class.getName()) + .warning( + "Invalid value for sysprop " + + MMapDirectory.SHARED_ARENA_MAX_PERMITS_SYSPROP + + ", must be positive and <= 0x07FF. The default value will be used."); + return RefCountedSharedArena.DEFAULT_MAX_PERMITS; + } + + /** + * Gets an arena for the given group, potentially aggregating files from the same segment into a + * single ref counted shared arena. A ref counted shared arena, if created will be added to the + * given arenas map. + */ + private Arena getSharedArena( + Optional group, ConcurrentHashMap arenas) { + if (group.isEmpty()) { + return Arena.ofShared(); + } + + String key = group.get(); + var refCountedArena = + arenas.computeIfAbsent( + key, s -> new RefCountedSharedArena(s, () -> arenas.remove(s), sharedArenaMaxPermits)); + if (refCountedArena.acquire()) { + return refCountedArena; + } else { + return arenas.compute( + key, + (s, v) -> { + if (v != null && v.acquire()) { + return v; + } else { + v = new RefCountedSharedArena(s, () -> arenas.remove(s), sharedArenaMaxPermits); + v.acquire(); // guaranteed to succeed + return v; + } + }); + } + } } diff --git a/lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java b/lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java new file mode 100644 index 00000000000..e89de280a3c --- /dev/null +++ b/lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java @@ -0,0 +1,146 @@ +/* + * 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.store; + +import java.lang.foreign.Arena; +import java.lang.foreign.MemorySegment; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A reference counted shared Arena. + * + *

The purpose of this class is to allow a number of mmapped memory segments to be associated + * with a single underlying arena in order to avoid closing the underlying arena until all segments + * are closed. Typically, these memory segments belong to the same logical group, e.g. individual + * files of the same index segment. We do this to avoid the expensive cost of closing a shared + * Arena. + * + *

The reference count is increased by {@link #acquire()}, and decreased by {@link #release()}. + * When the reference count reaches 0, then the underlying arena is closed and the given {@code + * onClose} runnable is executed. No more references can be acquired. + * + *

The total number of acquires that can be obtained for the lifetime of an instance of this + * class is 1024. When the total number of acquires is exhausted, then no more acquires are + * permitted and {@link #acquire()} returns false. This is independent of the actual number of the + * ref count. + */ +@SuppressWarnings("preview") +final class RefCountedSharedArena implements Arena { + + // default maximum permits + static final int DEFAULT_MAX_PERMITS = 1024; + + private static final int CLOSED = 0; + // minimum value, beyond which permits are exhausted + private static final int REMAINING_UNIT = 1 << 16; + // acquire decrement; effectively decrements permits and increments ref count + private static final int ACQUIRE_DECREMENT = REMAINING_UNIT - 1; // 0xffff + + private final String segmentName; + private final Runnable onClose; + private final Arena arena; + + // high 16 bits contain the total remaining acquires; monotonically decreasing + // low 16 bit contain the current ref count + private final AtomicInteger state; + + RefCountedSharedArena(String segmentName, Runnable onClose) { + this(segmentName, onClose, DEFAULT_MAX_PERMITS); + } + + RefCountedSharedArena(String segmentName, Runnable onClose, int maxPermits) { + if (validMaxPermits(maxPermits) == false) { + throw new IllegalArgumentException("invalid max permits: " + maxPermits); + } + this.segmentName = segmentName; + this.onClose = onClose; + this.arena = Arena.ofShared(); + this.state = new AtomicInteger(maxPermits << 16); + } + + static boolean validMaxPermits(int v) { + return v > 0 && v <= 0x7FFF; + } + + // for debugging + String getSegmentName() { + return segmentName; + } + + /** + * Returns true if the ref count has been increased. Otherwise, false if there are no remaining + * acquires. + */ + boolean acquire() { + int value; + while (true) { + value = state.get(); + if (value < REMAINING_UNIT) { + return false; + } + if (this.state.compareAndSet(value, value - ACQUIRE_DECREMENT)) { + return true; + } + } + } + + /** Decrements the ref count. */ + void release() { + int value; + while (true) { + value = state.get(); + final int count = value & 0xFFFF; + if (count == 0) { + throw new IllegalStateException(value == CLOSED ? "closed" : "nothing to release"); + } + final int newValue = count == 1 ? CLOSED : value - 1; + if (this.state.compareAndSet(value, newValue)) { + if (newValue == CLOSED) { + onClose.run(); + arena.close(); + } + return; + } + } + } + + @Override + public void close() { + release(); + } + + @Override + public MemorySegment allocate(long byteSize, long byteAlignment) { + throw new UnsupportedOperationException(); + } + + @Override + public MemorySegment.Scope scope() { + return arena.scope(); + } + + @Override + public String toString() { + return "RefCountedArena[segmentName=" + + segmentName + + ", value=" + + state.get() + + ", arena=" + + arena + + "]"; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java index f7c49c9b661..d01d6ec50eb 100644 --- a/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java +++ b/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java @@ -16,14 +16,22 @@ */ package org.apache.lucene.store; +import static java.util.stream.Collectors.toList; + import java.io.IOException; import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Random; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.function.Supplier; +import java.util.stream.IntStream; import org.apache.lucene.tests.store.BaseDirectoryTestCase; import org.apache.lucene.util.Constants; import org.apache.lucene.util.NamedThreadFactory; @@ -172,4 +180,153 @@ public class TestMMapDirectory extends BaseDirectoryTestCase { throw ee.getCause(); } } + + public void testArenas() throws Exception { + Supplier randomGenerationOrNone = + () -> random().nextBoolean() ? "_" + random().nextInt(5) : ""; + // First, create a number of segment specific file name lists to test with + var exts = + List.of( + ".si", ".cfs", ".cfe", ".dvd", ".dvm", ".nvd", ".nvm", ".fdt", ".vec", ".vex", ".vemf"); + var names = + IntStream.range(0, 50) + .mapToObj(i -> "_" + i + randomGenerationOrNone.get()) + .flatMap(s -> exts.stream().map(ext -> s + ext)) + .collect(toList()); + // Second, create a number of non-segment file names + IntStream.range(0, 50).mapToObj(i -> "foo" + i).forEach(names::add); + Collections.shuffle(names, random()); + + final int size = 6; + byte[] bytes = new byte[size]; + random().nextBytes(bytes); + + try (var dir = new MMapDirectory(createTempDir("testArenas"))) { + for (var name : names) { + try (IndexOutput out = dir.createOutput(name, IOContext.DEFAULT)) { + out.writeBytes(bytes, 0, bytes.length); + } + } + + int nThreads = 10; + int perListSize = (names.size() + nThreads) / nThreads; + List> nameLists = + IntStream.range(0, nThreads) + .mapToObj( + i -> + names.subList( + perListSize * i, Math.min(perListSize * i + perListSize, names.size()))) + .toList(); + + var threadFactory = new NamedThreadFactory("testArenas"); + try (var executor = Executors.newFixedThreadPool(nThreads, threadFactory)) { + var tasks = nameLists.stream().map(l -> new IndicesOpenTask(l, dir)).toList(); + var futures = tasks.stream().map(executor::submit).toList(); + for (var future : futures) { + future.get(); + } + } + + if (!(dir.attachment instanceof ConcurrentHashMap map)) { + throw new AssertionError("unexpected attachment: " + dir.attachment); + } + assertEquals(0, map.size()); + } + } + + static class IndicesOpenTask implements Callable { + final List names; + final Directory dir; + + IndicesOpenTask(List names, Directory dir) { + this.names = names; + this.dir = dir; + } + + @Override + public Void call() throws Exception { + List closeables = new ArrayList<>(); + for (var name : names) { + closeables.add(dir.openInput(name, IOContext.DEFAULT)); + } + for (IndexInput closeable : closeables) { + closeable.close(); + } + return null; + } + } + + // Opens more files in the same group than the ref counting limit. + public void testArenasManySegmentFiles() throws Exception { + var names = IntStream.range(0, 1024).mapToObj(i -> "_001.ext" + i).toList(); + + final int size = 4; + byte[] bytes = new byte[size]; + random().nextBytes(bytes); + + try (var dir = new MMapDirectory(createTempDir("testArenasManySegmentFiles"))) { + for (var name : names) { + try (IndexOutput out = dir.createOutput(name, IOContext.DEFAULT)) { + out.writeBytes(bytes, 0, bytes.length); + } + } + + List closeables = new ArrayList<>(); + for (var name : names) { + closeables.add(dir.openInput(name, IOContext.DEFAULT)); + } + for (IndexInput closeable : closeables) { + closeable.close(); + } + + if (!(dir.attachment instanceof ConcurrentHashMap map)) { + throw new AssertionError("unexpected attachment: " + dir.attachment); + } + assertEquals(0, map.size()); + } + } + + public void testGroupBySegmentFunc() { + var func = MMapDirectory.GROUP_BY_SEGMENT; + assertEquals("0", func.apply("_0.doc").orElseThrow()); + assertEquals("51", func.apply("_51.si").orElseThrow()); + assertEquals("51-g", func.apply("_51_1.si").orElseThrow()); + assertEquals("51-g", func.apply("_51_1_gg_ff.si").orElseThrow()); + assertEquals("51-g", func.apply("_51_2_gg_ff.si").orElseThrow()); + assertEquals("51-g", func.apply("_51_3_gg_ff.si").orElseThrow()); + assertEquals("5987654321", func.apply("_5987654321.si").orElseThrow()); + assertEquals("f", func.apply("_f.si").orElseThrow()); + assertEquals("ff", func.apply("_ff.si").orElseThrow()); + assertEquals("51a", func.apply("_51a.si").orElseThrow()); + assertEquals("f51a", func.apply("_f51a.si").orElseThrow()); + assertEquals("segment", func.apply("_segment.si").orElseThrow()); + + // old style + assertEquals("5", func.apply("_5_Lucene90FieldsIndex-doc_ids_0.tmp").orElseThrow()); + + assertFalse(func.apply("").isPresent()); + assertFalse(func.apply("_").isPresent()); + assertFalse(func.apply("_.si").isPresent()); + assertFalse(func.apply("foo").isPresent()); + assertFalse(func.apply("_foo").isPresent()); + assertFalse(func.apply("__foo").isPresent()); + assertFalse(func.apply("_segment").isPresent()); + assertFalse(func.apply("segment.si").isPresent()); + } + + public void testNoGroupingFunc() { + var func = MMapDirectory.NO_GROUPING; + assertFalse(func.apply("_0.doc").isPresent()); + assertFalse(func.apply("_0.si").isPresent()); + assertFalse(func.apply("_54.si").isPresent()); + assertFalse(func.apply("_ff.si").isPresent()); + assertFalse(func.apply("_.si").isPresent()); + assertFalse(func.apply("foo").isPresent()); + assertFalse(func.apply("_foo").isPresent()); + assertFalse(func.apply("__foo").isPresent()); + assertFalse(func.apply("_segment").isPresent()); + assertFalse(func.apply("_segment.si").isPresent()); + assertFalse(func.apply("segment.si").isPresent()); + assertFalse(func.apply("_51a.si").isPresent()); + } } From 8d4f7a6e99d2da802b7019247b0f8f305d71c024 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 25 Jul 2024 15:38:21 +0200 Subject: [PATCH 24/66] Bump the window size of disjunction from 2,048 to 4,096. (#13605) It's been pointed multiple times that a difference between Tantivy and Lucene is the fact that Tantivy uses windows of 4,096 docs when Lucene has a 2x smaller window size of 2,048 docs and that this might explain part of the performance difference. luceneutil suggests that bumping the window size to 4,096 does indeed improve performance for counting queries, but not for top-k queries. I'm still suggesting to bump the window size across the board to keep our disjunction scorer consistent. --- .../apache/lucene/search/BooleanScorer.java | 2 +- .../lucene/search/MaxScoreBulkScorer.java | 2 +- .../lucene/search/TestMaxScoreBulkScorer.java | 26 +++++++++---------- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index 919cc1680cf..e6e4f456bf3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -30,7 +30,7 @@ import org.apache.lucene.util.PriorityQueue; */ final class BooleanScorer extends BulkScorer { - static final int SHIFT = 11; + static final int SHIFT = 12; static final int SIZE = 1 << SHIFT; static final int MASK = SIZE - 1; static final int SET_SIZE = 1 << (SHIFT - 6); diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index bce02cbdae6..8786343ccec 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -25,7 +25,7 @@ import org.apache.lucene.util.MathUtil; final class MaxScoreBulkScorer extends BulkScorer { - static final int INNER_WINDOW_SIZE = 1 << 11; + static final int INNER_WINDOW_SIZE = 1 << 12; private final int maxDoc; // All scorers, sorted by increasing max score. diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreBulkScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreBulkScorer.java index 4c731201dc6..d7ccea69275 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreBulkScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestMaxScoreBulkScorer.java @@ -125,19 +125,19 @@ public class TestMaxScoreBulkScorer extends LuceneTestCase { assertEquals(2 + 1, scorer.score(), 0); break; case 1: - assertEquals(2048, doc); + assertEquals(4096, doc); assertEquals(2, scorer.score(), 0); break; case 2: - assertEquals(6144, doc); + assertEquals(12288, doc); assertEquals(2 + 1, scorer.score(), 0); break; case 3: - assertEquals(8192, doc); + assertEquals(16384, doc); assertEquals(1, scorer.score(), 0); break; case 4: - assertEquals(10240, doc); + assertEquals(20480, doc); assertEquals(1, scorer.score(), 0); break; default: @@ -195,13 +195,13 @@ public class TestMaxScoreBulkScorer extends LuceneTestCase { assertEquals(2 + 1, scorer.score(), 0); break; case 1: - assertEquals(2048, doc); + assertEquals(4096, doc); assertEquals(2, scorer.score(), 0); // simulate top-2 retrieval scorer.setMinCompetitiveScore(Math.nextUp(2)); break; case 2: - assertEquals(6144, doc); + assertEquals(12288, doc); assertEquals(2 + 1, scorer.score(), 0); scorer.setMinCompetitiveScore(Math.nextUp(2 + 1)); break; @@ -268,19 +268,19 @@ public class TestMaxScoreBulkScorer extends LuceneTestCase { assertEquals(2 + 1, scorer.score(), 0); break; case 1: - assertEquals(2048, doc); + assertEquals(4096, doc); assertEquals(2, scorer.score(), 0); break; case 2: - assertEquals(6144, doc); + assertEquals(12288, doc); assertEquals(2 + 1 + 3, scorer.score(), 0); break; case 3: - assertEquals(8192, doc); + assertEquals(16384, doc); assertEquals(1, scorer.score(), 0); break; case 4: - assertEquals(10240, doc); + assertEquals(20480, doc); assertEquals(1 + 3, scorer.score(), 0); break; default: @@ -346,18 +346,18 @@ public class TestMaxScoreBulkScorer extends LuceneTestCase { assertEquals(2 + 1, scorer.score(), 0); break; case 1: - assertEquals(2048, doc); + assertEquals(4096, doc); assertEquals(2, scorer.score(), 0); // simulate top-2 retrieval scorer.setMinCompetitiveScore(Math.nextUp(2)); break; case 2: - assertEquals(6144, doc); + assertEquals(12288, doc); assertEquals(2 + 1 + 3, scorer.score(), 0); scorer.setMinCompetitiveScore(Math.nextUp(2 + 1)); break; case 3: - assertEquals(10240, doc); + assertEquals(20480, doc); assertEquals(1 + 3, scorer.score(), 0); scorer.setMinCompetitiveScore(Math.nextUp(1 + 3)); break; From 481ca2d30f53ca3f0b13d083428b2bbcb1b6ae4c Mon Sep 17 00:00:00 2001 From: Peter Gromov Date: Sat, 27 Jul 2024 21:39:00 +0200 Subject: [PATCH 25/66] hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions (#13612) * hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions --- lucene/CHANGES.txt | 2 ++ .../analysis/hunspell/ModifyingSuggester.java | 9 ++++-- .../lucene/analysis/hunspell/Suggester.java | 28 +++++++++++++++---- .../analysis/hunspell/TestSpellChecking.java | 8 ++++++ 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index b761a4fa058..703cef58580 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -253,6 +253,8 @@ API Changes * GITHUB#13469: Expose FlatVectorsFormat as a first-class format; can be configured using a custom Codec. (Michael Sokolov) +* GITHUB#13612: Hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions. (Peter Gromov) + New Features --------------------- diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java index 218150cb416..aa63a5ec80b 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java @@ -31,6 +31,7 @@ class ModifyingSuggester { private final String misspelled; private final WordCase wordCase; private final FragmentChecker fragmentChecker; + private final boolean proceedPastRep; private final char[] tryChars; private final Hunspell speller; @@ -39,13 +40,15 @@ class ModifyingSuggester { LinkedHashSet result, String misspelled, WordCase wordCase, - FragmentChecker checker) { + FragmentChecker checker, + boolean proceedPastRep) { this.speller = speller; tryChars = speller.dictionary.tryChars.toCharArray(); this.result = result; this.misspelled = misspelled; this.wordCase = wordCase; fragmentChecker = checker; + this.proceedPastRep = proceedPastRep; } /** @@ -125,9 +128,9 @@ class ModifyingSuggester { boolean hasGoodSuggestions = trySuggestion(word.toUpperCase(Locale.ROOT)); GradedSuggestions repResult = tryRep(word); - if (repResult == GradedSuggestions.Best) return true; + if (repResult == GradedSuggestions.Best && !proceedPastRep) return true; - hasGoodSuggestions |= repResult == GradedSuggestions.Normal; + hasGoodSuggestions |= repResult != GradedSuggestions.None; if (!speller.dictionary.mapTable.isEmpty()) { enumerateMapReplacements(word, "", 0); diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java index e6da3c90584..6d4c12d71b5 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java @@ -53,16 +53,21 @@ public class Suggester { private final Dictionary dictionary; private final SuggestibleEntryCache suggestibleCache; private final FragmentChecker fragmentChecker; + private final boolean proceedPastRep; public Suggester(Dictionary dictionary) { - this(dictionary, null, FragmentChecker.EVERYTHING_POSSIBLE); + this(dictionary, null, FragmentChecker.EVERYTHING_POSSIBLE, false); } private Suggester( - Dictionary dictionary, SuggestibleEntryCache suggestibleCache, FragmentChecker checker) { + Dictionary dictionary, + SuggestibleEntryCache suggestibleCache, + FragmentChecker checker, + boolean proceedPastRep) { this.dictionary = dictionary; this.suggestibleCache = suggestibleCache; this.fragmentChecker = checker; + this.proceedPastRep = proceedPastRep; } /** @@ -71,8 +76,8 @@ public class Suggester { * entries are stored as fast-to-iterate plain words instead of highly compressed prefix trees. */ public Suggester withSuggestibleEntryCache() { - return new Suggester( - dictionary, SuggestibleEntryCache.buildCache(dictionary.words), fragmentChecker); + SuggestibleEntryCache cache = SuggestibleEntryCache.buildCache(dictionary.words); + return new Suggester(dictionary, cache, fragmentChecker, proceedPastRep); } /** @@ -80,7 +85,17 @@ public class Suggester { * the performance of the "Modification" phase performance. */ public Suggester withFragmentChecker(FragmentChecker checker) { - return new Suggester(dictionary, suggestibleCache, checker); + return new Suggester(dictionary, suggestibleCache, checker, proceedPastRep); + } + + /** + * Returns a copy of this suggester instance that doesn't stop after encountering acceptable words + * after applying REP rules. By default, Hunspell stops when it finds any, but this behavior may + * not always be desirable, e.g., if we have "REP i ea", "tims" be replaced only by "teams" and + * not "times", which could also be meant. + */ + public Suggester proceedPastRep() { + return new Suggester(dictionary, suggestibleCache, fragmentChecker, true); } /** @@ -174,7 +189,8 @@ public class Suggester { } boolean hasGoodSuggestions = - new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase, fragmentChecker) + new ModifyingSuggester( + suggestionSpeller, suggestions, word, wordCase, fragmentChecker, proceedPastRep) .suggest(); if (!hasGoodSuggestions && dictionary.maxNGramSuggestions > 0) { diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java index 4ca262799c6..6426fcc7a26 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java @@ -59,6 +59,14 @@ public class TestSpellChecking extends LuceneTestCase { public void testRepSuggestions() throws Exception { doTest("rep"); + + //noinspection DataFlowIssue + Path aff = Path.of(getClass().getResource("rep.aff").toURI()); + Dictionary dictionary = TestAllDictionaries.loadDictionary(aff); + Suggester suggester = new Suggester(dictionary); + assertEquals(List.of("auto's"), suggester.suggestNoTimeout("autos", () -> {})); + assertEquals( + List.of("auto's", "auto"), suggester.proceedPastRep().suggestNoTimeout("autos", () -> {})); } public void testPhSuggestions() throws Exception { From 8a7d4842cc7b96affea7c910769853f5b34e35b6 Mon Sep 17 00:00:00 2001 From: Jakub Slowinski <32519034+slow-J@users.noreply.github.com> Date: Mon, 29 Jul 2024 10:05:55 +0100 Subject: [PATCH 26/66] Remove usage of TopScoreDocCollector + TopFieldCollector deprecated methods (#create, #createSharedManager) (#13500) These methods were deprecated in #240 which is part of Lucene 10.0. Addresses #13499 --- lucene/CHANGES.txt | 2 ++ .../byTask/tasks/SearchWithCollectorTask.java | 5 +++-- .../lucene/search/TopFieldCollector.java | 4 ++-- .../TestBooleanQueryVisitSubscorers.java | 2 +- .../search/TestElevationComparator.java | 2 +- .../apache/lucene/search/TestNeedsScores.java | 2 +- .../TestPositiveScoresOnlyCollector.java | 4 +++- .../apache/lucene/search/TestSearchAfter.java | 20 ++++++----------- .../lucene/search/TestSortOptimization.java | 22 +++++++++---------- .../lucene/search/TestTopDocsMerge.java | 7 ++++-- .../apache/lucene/facet/FacetsCollector.java | 13 +++++------ .../grouping/BlockGroupingCollector.java | 13 ++++++----- .../search/grouping/TopGroupsCollector.java | 15 ++++++++----- .../lucene/search/join/TestJoinUtil.java | 4 ++-- 14 files changed, 61 insertions(+), 54 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 703cef58580..f749dc67d22 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -244,6 +244,8 @@ Other * GITHUB#13332: Improve MissingDoclet linter to check records correctly. (Uwe Schindler) +* GITHUB#13499: Remove usage of TopScoreDocCollector + TopFieldCollector deprecated methods (#create, #createSharedManager) (Jakub Slowinski) + ======================== Lucene 9.12.0 ======================= API Changes diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithCollectorTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithCollectorTask.java index 9f79e51d505..f41994f48d6 100644 --- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithCollectorTask.java +++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithCollectorTask.java @@ -20,7 +20,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData; import org.apache.lucene.benchmark.byTask.feeds.QueryMaker; import org.apache.lucene.benchmark.byTask.utils.Config; import org.apache.lucene.search.Collector; -import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.TopScoreDocCollectorManager; /** Does search w/ a custom collector */ public class SearchWithCollectorTask extends SearchTask { @@ -49,7 +49,8 @@ public class SearchWithCollectorTask extends SearchTask { protected Collector createCollector() throws Exception { Collector collector = null; if (clnName.equalsIgnoreCase("topScoreDoc") == true) { - collector = TopScoreDocCollector.create(numHits(), Integer.MAX_VALUE); + collector = + new TopScoreDocCollectorManager(numHits(), null, Integer.MAX_VALUE, false).newCollector(); } else if (clnName.length() > 0) { collector = Class.forName(clnName).asSubclass(Collector.class).getConstructor().newInstance(); diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index fcb14e1feef..1f4e840400f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -30,8 +30,8 @@ import org.apache.lucene.search.TotalHits.Relation; /** * A {@link Collector} that sorts by {@link SortField} using {@link FieldComparator}s. * - *

See the {@link #create(org.apache.lucene.search.Sort, int, int)} method for instantiating a - * TopFieldCollector. + *

See the constructor of {@link TopFieldCollectorManager} for instantiating a + * TopFieldCollectorManager with support for concurrency in IndexSearcher. * * @lucene.experimental */ diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java index 68c4c6f8f19..b291a96c802 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java @@ -156,7 +156,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase { private final Set tqsSet = new HashSet<>(); MyCollector() { - super(TopScoreDocCollector.create(10, Integer.MAX_VALUE)); + super(new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector()); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java index 075be5918e9..46ab7ad420b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java @@ -98,7 +98,7 @@ public class TestElevationComparator extends LuceneTestCase { TopDocs topDocs = searcher.search( - newq.build(), TopFieldCollector.createSharedManager(sort, 50, null, Integer.MAX_VALUE)); + newq.build(), new TopFieldCollectorManager(sort, 50, null, Integer.MAX_VALUE, true)); int nDocsReturned = topDocs.scoreDocs.length; assertEquals(4, nDocsReturned); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java b/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java index f36d2edfd97..b8392fefbdb 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java @@ -82,7 +82,7 @@ public class TestNeedsScores extends LuceneTestCase { TopDocs hits = searcher.search( - constantScore, TopScoreDocCollector.createSharedManager(5, null, Integer.MAX_VALUE)); + constantScore, new TopScoreDocCollectorManager(5, null, Integer.MAX_VALUE, true)); assertEquals(5, hits.totalHits.value); // Queries that support dynamic pruning like top-score or top-doc queries that do not compute diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java index c128945bd42..067c47db7f8 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java @@ -109,7 +109,9 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase { IndexReader ir = writer.getReader(); writer.close(); Scorer s = new SimpleScorer(); - TopDocsCollector tdc = TopScoreDocCollector.create(scores.length, Integer.MAX_VALUE); + TopDocsCollector tdc = + new TopScoreDocCollectorManager(scores.length, null, Integer.MAX_VALUE, false) + .newCollector(); Collector c = new PositiveScoresOnlyCollector(tdc); LeafCollector ac = c.getLeafCollector(ir.leaves().get(0)); ac.setScorer(s); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java b/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java index a1b7198fa3b..d0ecb7c1ad1 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java @@ -231,13 +231,13 @@ public class TestSearchAfter extends LuceneTestCase { final boolean doScores; final CollectorManager allManager; if (sort == null) { - allManager = TopScoreDocCollector.createSharedManager(maxDoc, null, Integer.MAX_VALUE); + allManager = new TopScoreDocCollectorManager(maxDoc, null, Integer.MAX_VALUE); doScores = false; } else if (sort == Sort.RELEVANCE) { - allManager = TopFieldCollector.createSharedManager(sort, maxDoc, null, Integer.MAX_VALUE); + allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true); doScores = true; } else { - allManager = TopFieldCollector.createSharedManager(sort, maxDoc, null, Integer.MAX_VALUE); + allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true); doScores = random().nextBoolean(); } all = searcher.search(query, allManager); @@ -269,20 +269,14 @@ public class TestSearchAfter extends LuceneTestCase { System.out.println(" iter lastBottom=" + lastBottom); } pagedManager = - TopScoreDocCollector.createSharedManager(pageSize, lastBottom, Integer.MAX_VALUE); + new TopScoreDocCollectorManager(pageSize, lastBottom, Integer.MAX_VALUE, true); } else { if (VERBOSE) { System.out.println(" iter lastBottom=" + lastBottom); } - if (sort == Sort.RELEVANCE) { - pagedManager = - TopFieldCollector.createSharedManager( - sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE); - } else { - pagedManager = - TopFieldCollector.createSharedManager( - sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE); - } + pagedManager = + new TopFieldCollectorManager( + sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE, true); } paged = searcher.search(query, pagedManager); if (doScores) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java index ad955e668ed..a1c0372cd08 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java @@ -237,7 +237,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField2.setMissingValue(0L); // set a competitive missing value final Sort sort = new Sort(sortField1, sortField2); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertEquals( @@ -264,7 +264,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value final Sort sort = new Sort(sortField); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs); @@ -279,7 +279,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value final Sort sort = new Sort(sortField); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs); @@ -323,7 +323,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(0L); // missing value is not competitive final Sort sort = new Sort(sortField); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); topDocs1 = searcher.search(new MatchAllDocsQuery(), manager); assertNonCompetitiveHitsAreSkipped(topDocs1.totalHits.value, numDocs); } @@ -334,7 +334,7 @@ public class TestSortOptimization extends LuceneTestCase { final Sort sort = new Sort(sortField); sortField.setOptimizeSortWithPoints(false); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); topDocs2 = searcher.search(new MatchAllDocsQuery(), manager); // assert that the resulting hits are the same assertEquals(topDocs1.scoreDocs.length, topDocs2.scoreDocs.length); @@ -357,7 +357,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField2.setMissingValue(0L); // missing value is not competitive final Sort multiSorts = new Sort(new SortField[] {sortField1, sortField2}); CollectorManager manager = - TopFieldCollector.createSharedManager(multiSorts, numHits, null, totalHitsThreshold); + new TopFieldCollectorManager(multiSorts, numHits, null, totalHitsThreshold, true); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); // can't optimization with NumericDocValues when there are multiple comparators assertEquals(topDocs.totalHits.value, numDocs); @@ -935,7 +935,7 @@ public class TestSortOptimization extends LuceneTestCase { // test search int numHits = 1 + random().nextInt(100); CollectorManager manager = - TopFieldCollector.createSharedManager(new Sort(sortField), numHits, null, numHits); + new TopFieldCollectorManager(new Sort(sortField), numHits, null, numHits, true); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); for (int i = 0; i < topDocs.scoreDocs.length; i++) { long expectedSeqNo = seqNos.get(i); @@ -986,12 +986,12 @@ public class TestSortOptimization extends LuceneTestCase { int expectedHits = Math.min(numDocs - visitedHits, batch); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, batch, (FieldDoc) after, totalHitsThreshold); + new TopFieldCollectorManager(sort, batch, (FieldDoc) after, totalHitsThreshold, true); TopDocs topDocs = searcher.search(query, manager); ScoreDoc[] scoreDocs = topDocs.scoreDocs; CollectorManager manager2 = - TopFieldCollector.createSharedManager(sort2, batch, (FieldDoc) after, totalHitsThreshold); + new TopFieldCollectorManager(sort2, batch, (FieldDoc) after, totalHitsThreshold, true); TopDocs topDocs2 = searcher.search(query, manager2); ScoreDoc[] scoreDocs2 = topDocs2.scoreDocs; @@ -1186,7 +1186,7 @@ public class TestSortOptimization extends LuceneTestCase { final int totalHitsThreshold = 5; CollectorManager manager = - TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); @@ -1216,7 +1216,7 @@ public class TestSortOptimization extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader, true, true, false); Query query = new MatchAllDocsQuery(); CollectorManager manager = - TopFieldCollector.createSharedManager(sort, n, after, n); + new TopFieldCollectorManager(sort, n, after, n, true); TopDocs topDocs = searcher.search(query, manager); IndexSearcher unoptimizedSearcher = newSearcher(new NoIndexDirectoryReader(reader), true, true, false); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java index e683b39c310..353fb66f358 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java @@ -55,7 +55,8 @@ public class TestTopDocsMerge extends LuceneTestCase { } public TopDocs search(Weight weight, int topN) throws IOException { - TopScoreDocCollector collector = TopScoreDocCollector.create(topN, Integer.MAX_VALUE); + TopScoreDocCollector collector = + new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE, false).newCollector(); search(ctx, weight, collector); return collector.topDocs(); } @@ -346,7 +347,9 @@ public class TestTopDocsMerge extends LuceneTestCase { if (sort == null) { subHits = subSearcher.search(w, numHits); } else { - final TopFieldCollector c = TopFieldCollector.create(sort, numHits, Integer.MAX_VALUE); + final TopFieldCollector c = + new TopFieldCollectorManager(sort, numHits, null, Integer.MAX_VALUE, false) + .newCollector(); subSearcher.search(w, c); subHits = c.topDocs(0, numHits); } diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java index 41db7b6df45..87ad2947cf5 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java @@ -34,8 +34,9 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopFieldCollectorManager; import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.TopScoreDocCollectorManager; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.ArrayUtil; @@ -253,13 +254,11 @@ public class FacetsCollector extends SimpleCollector { throw new IllegalArgumentException("after must be a FieldDoc; got " + after); } hitsCollector = - TopFieldCollector.create( - sort, - n, - (FieldDoc) after, - Integer.MAX_VALUE); // TODO: can we disable exact hit counts + new TopFieldCollectorManager(sort, n, (FieldDoc) after, Integer.MAX_VALUE, false) + .newCollector(); // TODO: can we disable exact hit counts } else { - hitsCollector = TopScoreDocCollector.create(n, after, Integer.MAX_VALUE); + hitsCollector = + new TopScoreDocCollectorManager(n, after, Integer.MAX_VALUE, false).newCollector(); } searcher.search(q, MultiCollector.wrap(hitsCollector, fc)); diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index 66972b827ed..92c420c5837 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -32,8 +32,8 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.TopFieldCollectorManager; +import org.apache.lucene.search.TopScoreDocCollectorManager; import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.Weight; import org.apache.lucene.util.ArrayUtil; @@ -294,12 +294,15 @@ public class BlockGroupingCollector extends SimpleCollector { throw new IllegalArgumentException( "cannot sort by relevance within group: needsScores=false"); } - collector = TopScoreDocCollector.create(maxDocsPerGroup, Integer.MAX_VALUE); + collector = + new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false) + .newCollector(); } else { // Sort by fields collector = - TopFieldCollector.create( - withinGroupSort, maxDocsPerGroup, Integer.MAX_VALUE); // TODO: disable exact counts? + new TopFieldCollectorManager( + withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false) + .newCollector(); // TODO: disable exact counts? } float groupMaxScore = needsScores ? Float.NEGATIVE_INFINITY : Float.NaN; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java index b3cbb25005c..00940be58ec 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java @@ -31,7 +31,8 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopScoreDocCollector; +import org.apache.lucene.search.TopFieldCollectorManager; +import org.apache.lucene.search.TopScoreDocCollectorManager; import org.apache.lucene.util.ArrayUtil; /** @@ -127,15 +128,17 @@ public class TopGroupsCollector extends SecondPassGroupingCollector { supplier = () -> new TopDocsAndMaxScoreCollector( - true, TopScoreDocCollector.create(maxDocsPerGroup, Integer.MAX_VALUE), null); + true, + new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false) + .newCollector(), + null); } else { supplier = () -> { TopFieldCollector topDocsCollector = - TopFieldCollector.create( - withinGroupSort, - maxDocsPerGroup, - Integer.MAX_VALUE); // TODO: disable exact counts? + new TopFieldCollectorManager( + withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false) + .newCollector(); // TODO: disable exact counts? MaxScoreCollector maxScoreCollector = getMaxScores ? new MaxScoreCollector() : null; return new TopDocsAndMaxScoreCollector(false, topDocsCollector, maxScoreCollector); }; diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java index 1e9fc55c598..972929e8118 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java @@ -468,7 +468,7 @@ public class TestJoinUtil extends LuceneTestCase { final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc()); final TopScoreDocCollector topScoreDocCollector = - TopScoreDocCollector.create(10, Integer.MAX_VALUE); + new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector(); indexSearcher.search( joinQuery, MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector)); assertBitSet(expectedResult, actualResult, indexSearcher); @@ -1546,7 +1546,7 @@ public class TestJoinUtil extends LuceneTestCase { // be also testing TopDocsCollector... final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc()); final TopScoreDocCollector topScoreDocCollector = - TopScoreDocCollector.create(10, Integer.MAX_VALUE); + new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector(); indexSearcher.search( joinQuery, MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector)); From 74865bb92a59421406e2caa58e02ef83e12fcb40 Mon Sep 17 00:00:00 2001 From: Jakub Slowinski <32519034+slow-J@users.noreply.github.com> Date: Tue, 30 Jul 2024 10:33:16 +0100 Subject: [PATCH 27/66] Removing all deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (#13617) These are already marked for deprecation in 9.x and we previously removed all internal use of these methods in 10.0. Closes #13499 --- lucene/CHANGES.txt | 2 + .../lucene/search/TopFieldCollector.java | 67 ------------------- .../lucene/search/TopScoreDocCollector.java | 56 ---------------- 3 files changed, 2 insertions(+), 123 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f749dc67d22..618f206ddec 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -246,6 +246,8 @@ Other * GITHUB#13499: Remove usage of TopScoreDocCollector + TopFieldCollector deprecated methods (#create, #createSharedManager) (Jakub Slowinski) +* GITHUB#13499: Removing all deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) + ======================== Lucene 9.12.0 ======================= API Changes diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index 1f4e840400f..7caf34c95f2 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -390,73 +390,6 @@ public abstract class TopFieldCollector extends TopDocsCollector { } } - /** - * Creates a new {@link TopFieldCollector} from the given arguments. - * - *

NOTE: The instances returned by this method pre-allocate a full array of length - * numHits. - * - * @param sort the sort criteria (SortFields). - * @param numHits the number of results to collect. - * @param totalHitsThreshold the number of docs to count accurately. If the query matches more - * than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other - * hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit - * count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit - * count accurate, but this will also make query processing slower. - * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria. - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopFieldCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThreshold) { - return new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, false) - .newCollector(); - } - - /** - * Creates a new {@link TopFieldCollector} from the given arguments. - * - *

NOTE: The instances returned by this method pre-allocate a full array of length - * numHits. - * - * @param sort the sort criteria (SortFields). - * @param numHits the number of results to collect. - * @param after only hits after this FieldDoc will be collected - * @param totalHitsThreshold the number of docs to count accurately. If the query matches more - * than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other - * hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit - * count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit - * count accurate, but this will also make query processing slower. Setting totalHitsThreshold - * less than {@link Integer#MAX_VALUE} instructs Lucene to skip non-competitive documents - * whenever possible. For numeric sort fields the skipping functionality works when the same - * field is indexed both with doc values and points. In this case, there is an assumption that - * the same data is stored in these points and doc values. - * @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria. - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopFieldCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static TopFieldCollector create( - Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) { - return new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, false) - .newCollector(); - } - - /** - * Create a CollectorManager which uses a shared hit counter to maintain number of hits and a - * shared {@link MaxScoreAccumulator} to propagate the minimum score accross segments if the - * primary sort is by relevancy. - * - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopFieldCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static CollectorManager createSharedManager( - Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) { - - return new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true); - } - /** * Populate {@link ScoreDoc#score scores} of the given {@code topDocs}. * diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java index 304eff2f11a..f74f087b7d5 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java @@ -191,62 +191,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { } } - /** - * Creates a new {@link TopScoreDocCollector} given the number of hits to collect and the number - * of hits to count accurately. - * - *

NOTE: If the total hit count of the top docs is less than or exactly {@code - * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link - * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower - * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate - * but will also likely make query processing slower. - * - *

NOTE: The instances returned by this method pre-allocate a full array of length - * numHits, and fill the array with sentinel objects. - * - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static TopScoreDocCollector create(int numHits, int totalHitsThreshold) { - return new TopScoreDocCollectorManager(numHits, null, totalHitsThreshold, false).newCollector(); - } - - /** - * Creates a new {@link TopScoreDocCollector} given the number of hits to collect, the bottom of - * the previous page, and the number of hits to count accurately. - * - *

NOTE: If the total hit count of the top docs is less than or exactly {@code - * totalHitsThreshold} then this value is accurate. On the other hand, if the {@link - * TopDocs#totalHits} value is greater than {@code totalHitsThreshold} then its value is a lower - * bound of the hit count. A value of {@link Integer#MAX_VALUE} will make the hit count accurate - * but will also likely make query processing slower. - * - *

NOTE: The instances returned by this method pre-allocate a full array of length - * numHits, and fill the array with sentinel objects. - * - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static TopScoreDocCollector create(int numHits, ScoreDoc after, int totalHitsThreshold) { - return new TopScoreDocCollectorManager(numHits, after, totalHitsThreshold, false) - .newCollector(); - } - - /** - * Create a CollectorManager which uses a shared hit counter to maintain number of hits and a - * shared {@link MaxScoreAccumulator} to propagate the minimum score accross segments - * - * @deprecated This method is deprecated in favor of the constructor of {@link - * TopScoreDocCollectorManager} due to its support for concurrency in IndexSearcher - */ - @Deprecated - public static CollectorManager createSharedManager( - int numHits, ScoreDoc after, int totalHitsThreshold) { - return new TopScoreDocCollectorManager(numHits, after, totalHitsThreshold, true); - } - int docBase; ScoreDoc pqTop; final HitsThresholdChecker hitsThresholdChecker; From 68aa629f6cea960800bd77b668819f67fa94eb42 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 30 Jul 2024 11:36:28 +0200 Subject: [PATCH 28/66] Move changes entry for #13499 to API changes --- lucene/CHANGES.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 618f206ddec..26c43bd5843 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -108,6 +108,8 @@ API Changes * GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand) +* GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) + New Features --------------------- @@ -246,8 +248,6 @@ Other * GITHUB#13499: Remove usage of TopScoreDocCollector + TopFieldCollector deprecated methods (#create, #createSharedManager) (Jakub Slowinski) -* GITHUB#13499: Removing all deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) - ======================== Lucene 9.12.0 ======================= API Changes From 30c965ea575a5e75c2bf724a340aa690d82f1ec5 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 30 Jul 2024 17:17:27 +0200 Subject: [PATCH 29/66] Introduce IndexSearcher#searchLeaf(LeafReaderContext, Weight, Collector) method (#13603) There's a couple of places in the codebase where we extend `IndexSearcher` to customize per leaf behaviour, and in order to do that, we need to override the entire search method that loops through the leaves. A good example is `ScorerIndexSearcher`. Adding a `searchLeaf` method that provides the per leaf behaviour makes those cases a little easier to deal with. --- lucene/CHANGES.txt | 4 + .../apache/lucene/search/IndexSearcher.java | 73 ++++++++++++------- .../lucene/search/TestTopDocsMerge.java | 10 +-- .../lucene/search/grouping/TestGrouping.java | 8 +- .../tests/search/ScorerIndexSearcher.java | 34 ++++----- 5 files changed, 73 insertions(+), 56 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 26c43bd5843..dfe8a5e5442 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -259,6 +259,10 @@ API Changes * GITHUB#13612: Hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions. (Peter Gromov) +* GITHUB#13603: Introduced `IndexSearcher#searchLeaf(LeafReaderContext, Weight, Collector)` protected method to + facilitate customizing per-leaf behavior of search without requiring to override + `search(LeafReaderContext[], Weight, Collector)` which requires overriding the entire loop across the leaves (Luca Cavanna) + New Features --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 8c0da9539eb..77d6edf34a0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -674,7 +674,8 @@ public class IndexSearcher { /** * Lower-level search API. * - *

{@link LeafCollector#collect(int)} is called for every document.
+ *

{@link #searchLeaf(LeafReaderContext, Weight, Collector)} is called for every leaf + * partition.
* *

NOTE: this method executes the searches on all given leaves exclusively. To search across * all the searchers leaves use {@link #leafContexts}. @@ -694,40 +695,56 @@ public class IndexSearcher { // threaded...? the Collector could be sync'd? // always use single thread: for (LeafReaderContext ctx : leaves) { // search each subreader - final LeafCollector leafCollector; + searchLeaf(ctx, weight, collector); + } + } + + /** + * Lower-level search API + * + *

{@link LeafCollector#collect(int)} is called for every document.
+ * + * @param ctx the leaf to execute the search against + * @param weight to match document + * @param collector to receive hits + * @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()} + * clauses. + */ + protected void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector) + throws IOException { + final LeafCollector leafCollector; + try { + leafCollector = collector.getLeafCollector(ctx); + } catch ( + @SuppressWarnings("unused") + CollectionTerminatedException e) { + // there is no doc of interest in this reader context + // continue with the following leaf + return; + } + ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx); + if (scorerSupplier != null) { + scorerSupplier.setTopLevelScoringClause(); + BulkScorer scorer = scorerSupplier.bulkScorer(); + if (queryTimeout != null) { + scorer = new TimeLimitingBulkScorer(scorer, queryTimeout); + } try { - leafCollector = collector.getLeafCollector(ctx); + scorer.score(leafCollector, ctx.reader().getLiveDocs()); } catch ( @SuppressWarnings("unused") CollectionTerminatedException e) { - // there is no doc of interest in this reader context + // collection was terminated prematurely // continue with the following leaf - continue; + } catch ( + @SuppressWarnings("unused") + TimeLimitingBulkScorer.TimeExceededException e) { + partialResult = true; } - ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx); - if (scorerSupplier != null) { - scorerSupplier.setTopLevelScoringClause(); - BulkScorer scorer = scorerSupplier.bulkScorer(); - if (queryTimeout != null) { - scorer = new TimeLimitingBulkScorer(scorer, queryTimeout); - } - try { - scorer.score(leafCollector, ctx.reader().getLiveDocs()); - } catch ( - @SuppressWarnings("unused") - CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf - } catch ( - @SuppressWarnings("unused") - TimeLimitingBulkScorer.TimeExceededException e) { - partialResult = true; - } - } - // Note: this is called if collection ran successfully, including the above special cases of - // CollectionTerminatedException and TimeExceededException, but no other exception. - leafCollector.finish(); } + // Note: this is called if collection ran successfully, including the above special cases of + // CollectionTerminatedException and TimeExceededException, but no other exception. + leafCollector.finish(); } /** diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java index 353fb66f358..c936c4fe0bb 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java @@ -43,27 +43,27 @@ import org.apache.lucene.util.BytesRef; public class TestTopDocsMerge extends LuceneTestCase { private static class ShardSearcher extends IndexSearcher { - private final List ctx; + private final LeafReaderContext ctx; public ShardSearcher(LeafReaderContext ctx, IndexReaderContext parent) { super(parent); - this.ctx = Collections.singletonList(ctx); + this.ctx = ctx; } public void search(Weight weight, Collector collector) throws IOException { - search(ctx, weight, collector); + searchLeaf(ctx, weight, collector); } public TopDocs search(Weight weight, int topN) throws IOException { TopScoreDocCollector collector = new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE, false).newCollector(); - search(ctx, weight, collector); + searchLeaf(ctx, weight, collector); return collector.topDocs(); } @Override public String toString() { - return "ShardSearcher(" + ctx.get(0) + ")"; + return "ShardSearcher(" + ctx + ")"; } } diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java index 436a4ca45a2..38493a01bb6 100644 --- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java +++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java @@ -1571,20 +1571,20 @@ public class TestGrouping extends LuceneTestCase { } private static class ShardSearcher extends IndexSearcher { - private final List ctx; + private final LeafReaderContext ctx; public ShardSearcher(LeafReaderContext ctx, IndexReaderContext parent) { super(parent); - this.ctx = Collections.singletonList(ctx); + this.ctx = ctx; } public void search(Weight weight, Collector collector) throws IOException { - search(ctx, weight, collector); + searchLeaf(ctx, weight, collector); } @Override public String toString() { - return "ShardSearcher(" + ctx.get(0).reader() + ")"; + return "ShardSearcher(" + ctx.reader() + ")"; } } } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java index 139c7a421c9..a77229ef68f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/ScorerIndexSearcher.java @@ -17,7 +17,6 @@ package org.apache.lucene.tests.search; import java.io.IOException; -import java.util.List; import java.util.concurrent.Executor; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; @@ -53,25 +52,22 @@ public class ScorerIndexSearcher extends IndexSearcher { } @Override - protected void search(List leaves, Weight weight, Collector collector) + protected void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector) throws IOException { - collector.setWeight(weight); - for (LeafReaderContext ctx : leaves) { // search each subreader - // we force the use of Scorer (not BulkScorer) to make sure - // that the scorer passed to LeafCollector.setScorer supports - // Scorer.getChildren - Scorer scorer = weight.scorer(ctx); - if (scorer != null) { - final DocIdSetIterator iterator = scorer.iterator(); - final LeafCollector leafCollector = collector.getLeafCollector(ctx); - leafCollector.setScorer(scorer); - final Bits liveDocs = ctx.reader().getLiveDocs(); - for (int doc = iterator.nextDoc(); - doc != DocIdSetIterator.NO_MORE_DOCS; - doc = iterator.nextDoc()) { - if (liveDocs == null || liveDocs.get(doc)) { - leafCollector.collect(doc); - } + // we force the use of Scorer (not BulkScorer) to make sure + // that the scorer passed to LeafCollector.setScorer supports + // Scorer.getChildren + Scorer scorer = weight.scorer(ctx); + if (scorer != null) { + final DocIdSetIterator iterator = scorer.iterator(); + final LeafCollector leafCollector = collector.getLeafCollector(ctx); + leafCollector.setScorer(scorer); + final Bits liveDocs = ctx.reader().getLiveDocs(); + for (int doc = iterator.nextDoc(); + doc != DocIdSetIterator.NO_MORE_DOCS; + doc = iterator.nextDoc()) { + if (liveDocs == null || liveDocs.get(doc)) { + leafCollector.collect(doc); } } } From 5226e282b4d69baf6803b856cd732a41aefc16e8 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 31 Jul 2024 17:11:44 +0200 Subject: [PATCH 30/66] Add test that the default codec parallelizes I/O. (#13579) This adds a Directory wrapper that counts how many times we wait for I/O to complete before doing something else, and adds tests that the default codec is able to parallelize I/O for stored fields retrieval and term lookups. --- .../index/TestDefaultCodecParallelizesIO.java | 104 +++++++++ .../store/SerialIOCountingDirectory.java | 210 ++++++++++++++++++ .../TestSerializedIOCountingDirectory.java | 101 +++++++++ 3 files changed, 415 insertions(+) create mode 100644 lucene/core/src/test/org/apache/lucene/index/TestDefaultCodecParallelizesIO.java create mode 100644 lucene/test-framework/src/java/org/apache/lucene/tests/store/SerialIOCountingDirectory.java create mode 100644 lucene/test-framework/src/test/org/apache/lucene/tests/store/TestSerializedIOCountingDirectory.java diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDefaultCodecParallelizesIO.java b/lucene/core/src/test/org/apache/lucene/index/TestDefaultCodecParallelizesIO.java new file mode 100644 index 00000000000..2c4351fa170 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestDefaultCodecParallelizesIO.java @@ -0,0 +1,104 @@ +/* + * 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.index; + +import java.io.IOException; +import org.apache.lucene.document.Document; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.store.SerialIOCountingDirectory; +import org.apache.lucene.tests.util.LineFileDocs; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOBooleanSupplier; +import org.apache.lucene.util.IOUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +public class TestDefaultCodecParallelizesIO extends LuceneTestCase { + + private static SerialIOCountingDirectory dir; + private static IndexReader reader; + + @BeforeClass + public static void beforeClass() throws Exception { + Directory bbDir = new ByteBuffersDirectory(); + try (LineFileDocs docs = new LineFileDocs(random()); + IndexWriter w = + new IndexWriter(bbDir, new IndexWriterConfig().setCodec(TestUtil.getDefaultCodec()))) { + final int numDocs = atLeast(10_000); + for (int d = 0; d < numDocs; ++d) { + Document doc = docs.nextDoc(); + w.addDocument(doc); + } + w.forceMerge(1); + } + dir = new SerialIOCountingDirectory(bbDir); + reader = DirectoryReader.open(dir); + } + + @AfterClass + public static void afterClass() throws Exception { + IOUtils.close(reader, dir); + } + + /** Simulate term lookup in a BooleanQuery. */ + public void testTermsSeekExact() throws IOException { + long prevCount = dir.count(); + + Terms terms = getOnlyLeafReader(reader).terms("body"); + String[] termValues = new String[] {"a", "which", "the", "for", "he"}; + IOBooleanSupplier[] suppliers = new IOBooleanSupplier[termValues.length]; + for (int i = 0; i < termValues.length; ++i) { + TermsEnum te = terms.iterator(); + suppliers[i] = te.prepareSeekExact(new BytesRef(termValues[i])); + } + int nonNullIOSuppliers = 0; + for (IOBooleanSupplier supplier : suppliers) { + if (supplier != null) { + nonNullIOSuppliers++; + supplier.get(); + } + } + + assertTrue(nonNullIOSuppliers > 0); + long newCount = dir.count(); + assertTrue(newCount - prevCount > 0); + assertTrue(newCount - prevCount < nonNullIOSuppliers); + } + + /** Simulate stored fields retrieval. */ + public void testStoredFields() throws IOException { + long prevCount = dir.count(); + + LeafReader leafReader = getOnlyLeafReader(reader); + StoredFields storedFields = leafReader.storedFields(); + int[] docs = new int[20]; + for (int i = 0; i < docs.length; ++i) { + docs[i] = random().nextInt(leafReader.maxDoc()); + storedFields.prefetch(docs[i]); + } + for (int doc : docs) { + storedFields.document(doc); + } + + long newCount = dir.count(); + assertTrue(newCount - prevCount > 0); + assertTrue(newCount - prevCount < docs.length); + } +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/store/SerialIOCountingDirectory.java b/lucene/test-framework/src/java/org/apache/lucene/tests/store/SerialIOCountingDirectory.java new file mode 100644 index 00000000000..2afbc2fd4a7 --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/store/SerialIOCountingDirectory.java @@ -0,0 +1,210 @@ +/* + * 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.tests.store; + +import java.io.IOException; +import java.util.concurrent.atomic.LongAdder; +import org.apache.lucene.internal.hppc.LongHashSet; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.ReadAdvice; +import org.apache.lucene.util.CloseableThreadLocal; + +/** + * A {@link Directory} wrapper that counts the number of times that Lucene may wait for I/O to + * return serially. Lower counts mean that Lucene better takes advantage of I/O parallelism. + */ +public class SerialIOCountingDirectory extends FilterDirectory { + + private static final long PAGE_SHIFT = 12; // 4096 bytes per page + // Assumed number of pages that are read ahead + private static final int PAGE_READAHEAD = 4; + private final LongAdder counter = new LongAdder(); + private final CloseableThreadLocal pendingFetch = + new CloseableThreadLocal() { + @Override + protected Boolean initialValue() { + return Boolean.FALSE; + } + }; + + /** Sole constructor. */ + public SerialIOCountingDirectory(Directory in) { + super(in); + } + + @Override + public void close() throws IOException { + pendingFetch.close(); + super.close(); + } + + /** Return the number of I/O request performed serially. */ + public long count() { + return counter.sum(); + } + + @Override + public ChecksumIndexInput openChecksumInput(String name) throws IOException { + // sequential access, count 1 for the whole file + counter.increment(); + return super.openChecksumInput(name); + } + + @Override + public IndexInput openInput(String name, IOContext context) throws IOException { + if (context.readAdvice() == ReadAdvice.RANDOM_PRELOAD) { + // expected to be loaded in memory, only count 1 at open time + counter.increment(); + return super.openInput(name, context); + } + return new SerializedIOCountingIndexInput(super.openInput(name, context), context.readAdvice()); + } + + private class SerializedIOCountingIndexInput extends IndexInput { + + private final IndexInput in; + private final long sliceOffset, sliceLength; + private final ReadAdvice readAdvice; + private final LongHashSet pendingPages = new LongHashSet(); + private long currentPage = Long.MIN_VALUE; + + public SerializedIOCountingIndexInput(IndexInput in, ReadAdvice readAdvice) { + this(in, readAdvice, 0L, in.length()); + } + + public SerializedIOCountingIndexInput( + IndexInput in, ReadAdvice readAdvice, long offset, long length) { + super(in.toString()); + this.in = in; + this.sliceOffset = offset; + this.sliceLength = length; + this.readAdvice = readAdvice; + } + + private void onRead(long offset, int len) { + if (len == 0) { + return; + } + final long firstPage = (sliceOffset + offset) >> PAGE_SHIFT; + final long lastPage = (sliceOffset + offset + len - 1) >> PAGE_SHIFT; + + for (long page = firstPage; page <= lastPage; ++page) { + long readAheadUpto; + if (readAdvice == ReadAdvice.RANDOM) { + readAheadUpto = currentPage; + } else { + // Assume that the next few pages are always free to read thanks to read-ahead. + readAheadUpto = currentPage + PAGE_READAHEAD; + } + + if (pendingPages.contains(page) == false && (page < currentPage || page > readAheadUpto)) { + counter.increment(); + } + currentPage = page; + } + pendingFetch.set(false); + } + + @Override + public void prefetch(long offset, long length) throws IOException { + final long firstPage = (sliceOffset + offset) >> PAGE_SHIFT; + final long lastPage = (sliceOffset + offset + length - 1) >> PAGE_SHIFT; + + long readAheadUpto; + if (readAdvice == ReadAdvice.RANDOM) { + readAheadUpto = currentPage; + } else { + // Assume that the next few pages are always free to read thanks to read-ahead. + readAheadUpto = currentPage + PAGE_READAHEAD; + } + + if (firstPage >= currentPage && lastPage <= readAheadUpto) { + // seeking within the current (or next page if ReadAdvice.NORMAL) doesn't increment the + // counter + } else if (pendingFetch.get() == false) { + // If multiple prefetch calls are performed without a readXXX() call in-between, count a + // single increment as these I/O requests can be performed in parallel. + counter.increment(); + pendingPages.clear(); + pendingFetch.set(true); + } + + for (long page = firstPage; page <= lastPage; ++page) { + pendingPages.add(page); + } + } + + @Override + public byte readByte() throws IOException { + onRead(getFilePointer(), Byte.BYTES); + return in.readByte(); + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException { + onRead(getFilePointer(), len); + in.readBytes(b, offset, len); + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public long getFilePointer() { + return in.getFilePointer() - sliceOffset; + } + + @Override + public void seek(long pos) throws IOException { + in.seek(sliceOffset + pos); + } + + @Override + public long length() { + return sliceLength; + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { + return slice(sliceDescription, offset, length, readAdvice); + } + + @Override + public IndexInput slice( + String sliceDescription, long offset, long length, ReadAdvice readAdvice) + throws IOException { + if (offset < 0 || offset + length > sliceLength) { + throw new IllegalArgumentException(); + } + IndexInput clone = in.clone(); + clone.seek(sliceOffset + offset); + return new SerializedIOCountingIndexInput(clone, readAdvice, sliceOffset + offset, length); + } + + @Override + public IndexInput clone() { + IndexInput clone = in.clone(); + return new SerializedIOCountingIndexInput(clone, readAdvice, sliceOffset, sliceLength); + } + } +} diff --git a/lucene/test-framework/src/test/org/apache/lucene/tests/store/TestSerializedIOCountingDirectory.java b/lucene/test-framework/src/test/org/apache/lucene/tests/store/TestSerializedIOCountingDirectory.java new file mode 100644 index 00000000000..7c6f232aef0 --- /dev/null +++ b/lucene/test-framework/src/test/org/apache/lucene/tests/store/TestSerializedIOCountingDirectory.java @@ -0,0 +1,101 @@ +/* + * 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.tests.store; + +import java.io.IOException; +import java.nio.file.Path; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.ReadAdvice; + +public class TestSerializedIOCountingDirectory extends BaseDirectoryTestCase { + + @Override + protected Directory getDirectory(Path path) throws IOException { + return new SerialIOCountingDirectory(FSDirectory.open(path)); + } + + public void testSequentialReads() throws IOException { + try (SerialIOCountingDirectory dir = new SerialIOCountingDirectory(newDirectory())) { + try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) { + for (int i = 0; i < 10; ++i) { + out.writeBytes(new byte[4096], 4096); + } + } + try (IndexInput in = + dir.openInput("test", IOContext.DEFAULT.withReadAdvice(ReadAdvice.NORMAL))) { + in.readByte(); + long count = dir.count(); + while (in.getFilePointer() < in.length()) { + in.readByte(); + } + // Sequential reads are free with the normal advice + assertEquals(count, dir.count()); + } + try (IndexInput in = + dir.openInput("test", IOContext.DEFAULT.withReadAdvice(ReadAdvice.RANDOM))) { + in.readByte(); + long count = dir.count(); + while (in.getFilePointer() < in.length()) { + in.readByte(); + } + // But not with the random advice + assertFalse(count == dir.count()); + } + } + } + + public void testParallelReads() throws IOException { + try (SerialIOCountingDirectory dir = new SerialIOCountingDirectory(newDirectory())) { + try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) { + for (int i = 0; i < 10; ++i) { + out.writeBytes(new byte[4096], 4096); + } + } + try (IndexInput in = + dir.openInput("test", IOContext.DEFAULT.withReadAdvice(ReadAdvice.RANDOM))) { + long count = dir.count(); + + // count is incremented on the first prefetch + in.prefetch(5_000, 1); + assertEquals(count + 1, dir.count()); + count = dir.count(); + + // but not on the second one since it can be performed in parallel + in.prefetch(10_000, 1); + assertEquals(count, dir.count()); + + // and reading from a prefetched page doesn't increment the counter + in.seek(5_000); + in.readByte(); + assertEquals(count, dir.count()); + + in.seek(10_000); + in.readByte(); + assertEquals(count, dir.count()); + + // reading data on a page that was not prefetched increments the counter + in.seek(15_000); + in.readByte(); + assertEquals(count + 1, dir.count()); + } + } + } +} From b4a8810b7aea2fa6143aa0323f924f85c5cb3329 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 31 Jul 2024 17:18:28 +0200 Subject: [PATCH 31/66] Inline skip data into postings lists (#13585) This updates the postings format in order to inline skip data into postings. This format is generally similar to the current `Lucene99PostingsFormat`, e.g. it shares the same block encoding logic, but it has a few differences: - Skip data is inlined into postings to make the access pattern more sequential. - There are only 2 levels of skip data: on every block (128 docs) and every 32 blocks (4,096 docs). In general, I found that the fact that skip data is inlined may slow down a bit queries that don't need skip data at all (e.g. `CountOrXXX` tasks that never advance of consult impacts) and speed up a bit queries that advance by small intervals. The fact that the greatest level only allows skipping 4096 docs at once means that we're slower at advancing by large intervals, but data suggests that it doesn't significantly hurt performance. --- gradle/generation/forUtil.gradle | 27 +- lucene/CHANGES.txt | 6 + .../checksums/generateForUtil99.json | 4 + .../backward-codecs/src/java/module-info.java | 7 +- .../lucene99/ForDeltaUtil.java | 2 +- .../backward_codecs/lucene99/ForUtil.java | 1148 +++++++++ .../lucene99/Lucene99Codec.java | 29 +- .../lucene99/Lucene99PostingsFormat.java | 38 +- .../lucene99/Lucene99PostingsReader.java | 20 +- .../lucene99/Lucene99ScoreSkipReader.java | 2 +- .../lucene99/Lucene99SkipReader.java | 11 +- .../lucene99/Lucene99SkipWriter.java | 5 +- .../backward_codecs/lucene99/PForUtil.java | 134 ++ .../lucene99/PostingsUtil.java | 2 +- .../backward_codecs/lucene99/gen_ForUtil.py | 524 ++++ .../lucene99/package-info.java | 428 ++++ .../services/org.apache.lucene.codecs.Codec | 1 + .../org.apache.lucene.codecs.PostingsFormat | 1 + .../lucene90/TestLucene90PostingsFormat.java | 19 +- .../lucene99/Lucene99PostingsWriter.java | 18 +- .../lucene99/Lucene99RWPostingsFormat.java | 68 + .../lucene99/TestForDeltaUtil.java | 2 +- .../lucene99/TestForUtil.java | 2 +- ...ene99HnswScalarQuantizedVectorsFormat.java | 1 - .../lucene99/TestLucene99PostingsFormat.java | 16 +- .../lucene99/TestPForUtil.java | 2 +- .../lucene99/TestPostingsUtil.java | 2 +- .../TestInt8HnswBackwardsCompatibility.java | 2 +- .../BlockTreeOrdsPostingsFormat.java | 10 +- .../codecs/memory/DirectPostingsFormat.java | 8 +- .../codecs/memory/FSTPostingsFormat.java | 8 +- .../DeltaBaseTermStateSerializer.java | 21 +- .../UniformSplitPostingsFormat.java | 8 +- .../codecs/uniformsplit/package-info.java | 2 +- .../bitvectors/TestHnswBitVectorsFormat.java | 4 +- .../lucene90/tests/MockTermStateFactory.java | 2 +- .../generated/checksums/generateForUtil.json | 4 +- lucene/core/src/java/module-info.java | 7 +- .../java/org/apache/lucene/codecs/Codec.java | 2 +- .../codecs/CompetitiveImpactAccumulator.java | 6 +- .../lucene90/Lucene90StoredFieldsFormat.java | 4 +- .../lucene/codecs/lucene912/ForDeltaUtil.java | 83 + .../{lucene99 => lucene912}/ForUtil.java | 2 +- .../codecs/lucene912/Lucene912Codec.java | 217 ++ .../lucene912/Lucene912PostingsFormat.java | 492 ++++ .../lucene912/Lucene912PostingsReader.java | 2104 +++++++++++++++++ .../lucene912/Lucene912PostingsWriter.java | 681 ++++++ .../{lucene99 => lucene912}/PForUtil.java | 2 +- .../lucene/codecs/lucene912/PostingsUtil.java | 73 + .../{lucene99 => lucene912}/gen_ForUtil.py | 2 +- .../lucene/codecs/lucene912/package-info.java | 431 ++++ .../lucene/codecs/lucene99/package-info.java | 411 +--- .../org/apache/lucene/search/PhraseQuery.java | 10 +- .../services/org.apache.lucene.codecs.Codec | 2 +- .../org.apache.lucene.codecs.PostingsFormat | 2 +- ...ne90StoredFieldsFormatHighCompression.java | 8 +- .../TestLucene912PostingsFormat.java | 157 ++ ...estLucene99HnswQuantizedVectorsFormat.java | 5 +- ...stLucene99ScalarQuantizedVectorScorer.java | 3 +- ...tLucene99ScalarQuantizedVectorsFormat.java | 3 +- .../lucene/index/TestLazyProxSkipping.java | 195 -- .../intervals/TermIntervalsSource.java | 10 +- lucene/suggest/src/java/module-info.java | 3 +- .../document/Completion912PostingsFormat.java | 45 + .../document/Completion99PostingsFormat.java | 6 +- .../org.apache.lucene.codecs.PostingsFormat | 1 + .../suggest/document/TestSuggestField.java | 2 +- .../codecs/blockterms/LuceneFixedGap.java | 12 +- .../LuceneVarGapDocFreqInterval.java | 12 +- .../blockterms/LuceneVarGapFixedInterval.java | 12 +- .../mockrandom/MockRandomPostingsFormat.java | 8 +- .../UniformSplitRot13PostingsFormat.java | 8 +- .../util/TestRuleSetupAndRestoreClassEnv.java | 8 +- .../apache/lucene/tests/util/TestUtil.java | 10 +- 74 files changed, 6816 insertions(+), 811 deletions(-) create mode 100644 lucene/backward-codecs/src/generated/checksums/generateForUtil99.json rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/ForDeltaUtil.java (98%) create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99Codec.java (82%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99PostingsFormat.java (95%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99PostingsReader.java (98%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99ScoreSkipReader.java (98%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99SkipReader.java (92%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/Lucene99SkipWriter.java (97%) create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene99/PostingsUtil.java (97%) create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py create mode 100644 lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/Lucene99PostingsWriter.java (95%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/TestForDeltaUtil.java (98%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/TestForUtil.java (98%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/TestLucene99PostingsFormat.java (90%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/TestPForUtil.java (98%) rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene99/TestPostingsUtil.java (97%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java rename lucene/core/src/java/org/apache/lucene/codecs/{lucene99 => lucene912}/ForUtil.java (99%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java rename lucene/core/src/java/org/apache/lucene/codecs/{lucene99 => lucene912}/PForUtil.java (99%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java rename lucene/core/src/java/org/apache/lucene/codecs/{lucene99 => lucene912}/gen_ForUtil.py (99%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java delete mode 100644 lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java create mode 100644 lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java diff --git a/gradle/generation/forUtil.gradle b/gradle/generation/forUtil.gradle index e1428234875..5de1d850a02 100644 --- a/gradle/generation/forUtil.gradle +++ b/gradle/generation/forUtil.gradle @@ -23,7 +23,7 @@ configure(project(":lucene:core")) { description "Regenerate gen_ForUtil.py" group "generation" - def genDir = file("src/java/org/apache/lucene/codecs/lucene99") + def genDir = file("src/java/org/apache/lucene/codecs/lucene912") def genScript = file("${genDir}/gen_ForUtil.py") def genOutput = file("${genDir}/ForUtil.java") @@ -96,5 +96,30 @@ configure(project(":lucene:backward-codecs")) { andThenTasks: ["spotlessJava", "spotlessJavaApply"], mustRunBefore: [ "compileJava" ] ]) + + task generateForUtil99Internal() { + description "Regenerate gen_ForUtil.py" + group "generation" + + def genDir = file("src/java/org/apache/lucene/backward_codecs/lucene99") + def genScript = file("${genDir}/gen_ForUtil.py") + def genOutput = file("${genDir}/ForUtil.java") + + inputs.file genScript + outputs.file genOutput + + doLast { + quietExec { + workingDir genDir + executable project.externalTool("python3") + args = [ '-B', genScript ] + } + } + } + + regenerate.dependsOn wrapWithPersistentChecksums(generateForUtil99Internal, [ + andThenTasks: ["spotlessJava", "spotlessJavaApply"], + mustRunBefore: [ "compileJava" ] + ]) } diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index dfe8a5e5442..d23c491fe32 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -317,6 +317,12 @@ Optimizations by 1) using a confined Arena where appropriate, and 2) grouping files from the same segment to a single shared Arena. (Chris Hegarty, Michael Gibney, Uwe Schindler) +* GITHUB#13585: Lucene912PostingsFormat, the new default postings format, now + only has 2 levels of skip data, which are inlined into postings instead of + being stored at the end of postings lists. This translates into better + performance for queries that need skipping such as conjunctions. + (Adrien Grand) + Changes in runtime behavior --------------------- diff --git a/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json b/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json new file mode 100644 index 00000000000..872e9c7bd6d --- /dev/null +++ b/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json @@ -0,0 +1,4 @@ +{ + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java": "f31797842f047626df6a1a6b97167bec60269fec", + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py": "325f2610974b0e76e278b6445405a098a3763feb" +} \ No newline at end of file diff --git a/lucene/backward-codecs/src/java/module-info.java b/lucene/backward-codecs/src/java/module-info.java index 992ad22a773..fbc2cdba98e 100644 --- a/lucene/backward-codecs/src/java/module-info.java +++ b/lucene/backward-codecs/src/java/module-info.java @@ -35,6 +35,7 @@ module org.apache.lucene.backward_codecs { exports org.apache.lucene.backward_codecs.lucene92; exports org.apache.lucene.backward_codecs.lucene94; exports org.apache.lucene.backward_codecs.lucene95; + exports org.apache.lucene.backward_codecs.lucene99; exports org.apache.lucene.backward_codecs.packed; exports org.apache.lucene.backward_codecs.store; @@ -43,7 +44,8 @@ module org.apache.lucene.backward_codecs { provides org.apache.lucene.codecs.PostingsFormat with org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat, org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat, - org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat; + org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat, + org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat, org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat, @@ -59,5 +61,6 @@ module org.apache.lucene.backward_codecs { org.apache.lucene.backward_codecs.lucene91.Lucene91Codec, org.apache.lucene.backward_codecs.lucene92.Lucene92Codec, org.apache.lucene.backward_codecs.lucene94.Lucene94Codec, - org.apache.lucene.backward_codecs.lucene95.Lucene95Codec; + org.apache.lucene.backward_codecs.lucene95.Lucene95Codec, + org.apache.lucene.backward_codecs.lucene99.Lucene99Codec; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java index 2403c997435..2fa67ebf40e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java new file mode 100644 index 00000000000..cc089751784 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java @@ -0,0 +1,1148 @@ +// This file has been automatically generated, DO NOT EDIT + +/* + * 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.backward_codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; + +// Inspired from https://fulmicoton.com/posts/bitpacking/ +// Encodes multiple integers in a long to get SIMD-like speedups. +// If bitsPerValue <= 8 then we pack 8 ints per long +// else if bitsPerValue <= 16 we pack 4 ints per long +// else we pack 2 ints per long +final class ForUtil { + + static final int BLOCK_SIZE = 128; + private static final int BLOCK_SIZE_LOG2 = 7; + + private static long expandMask32(long mask32) { + return mask32 | (mask32 << 32); + } + + private static long expandMask16(long mask16) { + return expandMask32(mask16 | (mask16 << 16)); + } + + private static long expandMask8(long mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + private static long mask32(int bitsPerValue) { + return expandMask32((1L << bitsPerValue) - 1); + } + + private static long mask16(int bitsPerValue) { + return expandMask16((1L << bitsPerValue) - 1); + } + + private static long mask8(int bitsPerValue) { + return expandMask8((1L << bitsPerValue) - 1); + } + + private static void expand8(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 56) & 0xFFL; + arr[16 + i] = (l >>> 48) & 0xFFL; + arr[32 + i] = (l >>> 40) & 0xFFL; + arr[48 + i] = (l >>> 32) & 0xFFL; + arr[64 + i] = (l >>> 24) & 0xFFL; + arr[80 + i] = (l >>> 16) & 0xFFL; + arr[96 + i] = (l >>> 8) & 0xFFL; + arr[112 + i] = l & 0xFFL; + } + } + + private static void expand8To32(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 24) & 0x000000FF000000FFL; + arr[16 + i] = (l >>> 16) & 0x000000FF000000FFL; + arr[32 + i] = (l >>> 8) & 0x000000FF000000FFL; + arr[48 + i] = l & 0x000000FF000000FFL; + } + } + + private static void collapse8(long[] arr) { + for (int i = 0; i < 16; ++i) { + arr[i] = + (arr[i] << 56) + | (arr[16 + i] << 48) + | (arr[32 + i] << 40) + | (arr[48 + i] << 32) + | (arr[64 + i] << 24) + | (arr[80 + i] << 16) + | (arr[96 + i] << 8) + | arr[112 + i]; + } + } + + private static void expand16(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 48) & 0xFFFFL; + arr[32 + i] = (l >>> 32) & 0xFFFFL; + arr[64 + i] = (l >>> 16) & 0xFFFFL; + arr[96 + i] = l & 0xFFFFL; + } + } + + private static void expand16To32(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL; + arr[32 + i] = l & 0x0000FFFF0000FFFFL; + } + } + + private static void collapse16(long[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = (arr[i] << 48) | (arr[32 + i] << 32) | (arr[64 + i] << 16) | arr[96 + i]; + } + } + + private static void expand32(long[] arr) { + for (int i = 0; i < 64; ++i) { + long l = arr[i]; + arr[i] = l >>> 32; + arr[64 + i] = l & 0xFFFFFFFFL; + } + } + + private static void collapse32(long[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 32) | arr[64 + i]; + } + } + + private static void prefixSum8(long[] arr, long base) { + expand8To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum16(long[] arr, long base) { + // We need to move to the next primitive size to avoid overflows + expand16To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum32(long[] arr, long base) { + arr[0] += base << 32; + innerPrefixSum32(arr); + expand32(arr); + final long l = arr[BLOCK_SIZE / 2 - 1]; + for (int i = BLOCK_SIZE / 2; i < BLOCK_SIZE; ++i) { + arr[i] += l; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum32(long[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + arr[32] += arr[31]; + arr[33] += arr[32]; + arr[34] += arr[33]; + arr[35] += arr[34]; + arr[36] += arr[35]; + arr[37] += arr[36]; + arr[38] += arr[37]; + arr[39] += arr[38]; + arr[40] += arr[39]; + arr[41] += arr[40]; + arr[42] += arr[41]; + arr[43] += arr[42]; + arr[44] += arr[43]; + arr[45] += arr[44]; + arr[46] += arr[45]; + arr[47] += arr[46]; + arr[48] += arr[47]; + arr[49] += arr[48]; + arr[50] += arr[49]; + arr[51] += arr[50]; + arr[52] += arr[51]; + arr[53] += arr[52]; + arr[54] += arr[53]; + arr[55] += arr[54]; + arr[56] += arr[55]; + arr[57] += arr[56]; + arr[58] += arr[57]; + arr[59] += arr[58]; + arr[60] += arr[59]; + arr[61] += arr[60]; + arr[62] += arr[61]; + arr[63] += arr[62]; + } + + private final long[] tmp = new long[BLOCK_SIZE / 2]; + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + final int numLongs; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + numLongs = BLOCK_SIZE / 8; + collapse8(longs); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + numLongs = BLOCK_SIZE / 4; + collapse16(longs); + } else { + nextPrimitive = 32; + numLongs = BLOCK_SIZE / 2; + collapse32(longs); + } + + final int numLongsPerShift = bitsPerValue * 2; + int idx = 0; + int shift = nextPrimitive - bitsPerValue; + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] = longs[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] |= longs[idx++] << shift; + } + } + + final int remainingBitsPerLong = shift + bitsPerValue; + final long maskRemainingBitsPerLong; + if (nextPrimitive == 8) { + maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong]; + } else if (nextPrimitive == 16) { + maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong]; + } else { + maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numLongs) { + if (remainingBitsPerValue >= remainingBitsPerLong) { + remainingBitsPerValue -= remainingBitsPerLong; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final long mask1, mask2; + if (nextPrimitive == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue]; + } else if (nextPrimitive == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numLongsPerShift; ++i) { + out.writeLong(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + throws IOException { + final int numLongs = bitsPerValue << 1; + in.readLongs(tmp, 0, numLongs); + final long mask = MASKS32[bitsPerValue]; + int longsIdx = 0; + int shift = 32 - bitsPerValue; + for (; shift >= 0; shift -= bitsPerValue) { + shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); + longsIdx += numLongs; + } + final int remainingBitsPerLong = shift + bitsPerValue; + final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerLong; + for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) { + int b = bitsPerValue - remainingBits; + long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerLong) { + b -= remainingBitsPerLong; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerLong - b)) & MASKS32[b]; + remainingBits = remainingBitsPerLong - b; + } else { + remainingBits = remainingBitsPerLong; + } + longs[longsIdx] = l; + } + } + + /** + * The pattern that this shiftLongs method applies is recognized by the C2 compiler, which + * generates SIMD instructions for it in order to shift multiple longs at once. + */ + private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = (a[i] >>> shift) & mask; + } + } + + private static final long[] MASKS8 = new long[8]; + private static final long[] MASKS16 = new long[16]; + private static final long[] MASKS32 = new long[32]; + + static { + for (int i = 0; i < 8; ++i) { + MASKS8[i] = mask8(i); + } + for (int i = 0; i < 16; ++i) { + MASKS16[i] = mask16(i); + } + for (int i = 0; i < 32; ++i) { + MASKS32[i] = mask32(i); + } + } + + // mark values in array as final longs to avoid the cost of reading array, arrays should only be + // used when the idx is a variable + private static final long MASK8_1 = MASKS8[1]; + private static final long MASK8_2 = MASKS8[2]; + private static final long MASK8_3 = MASKS8[3]; + private static final long MASK8_4 = MASKS8[4]; + private static final long MASK8_5 = MASKS8[5]; + private static final long MASK8_6 = MASKS8[6]; + private static final long MASK8_7 = MASKS8[7]; + private static final long MASK16_1 = MASKS16[1]; + private static final long MASK16_2 = MASKS16[2]; + private static final long MASK16_3 = MASKS16[3]; + private static final long MASK16_4 = MASKS16[4]; + private static final long MASK16_5 = MASKS16[5]; + private static final long MASK16_6 = MASKS16[6]; + private static final long MASK16_7 = MASKS16[7]; + private static final long MASK16_9 = MASKS16[9]; + private static final long MASK16_10 = MASKS16[10]; + private static final long MASK16_11 = MASKS16[11]; + private static final long MASK16_12 = MASKS16[12]; + private static final long MASK16_13 = MASKS16[13]; + private static final long MASK16_14 = MASKS16[14]; + private static final long MASK16_15 = MASKS16[15]; + private static final long MASK32_1 = MASKS32[1]; + private static final long MASK32_2 = MASKS32[2]; + private static final long MASK32_3 = MASKS32[3]; + private static final long MASK32_4 = MASKS32[4]; + private static final long MASK32_5 = MASKS32[5]; + private static final long MASK32_6 = MASKS32[6]; + private static final long MASK32_7 = MASKS32[7]; + private static final long MASK32_8 = MASKS32[8]; + private static final long MASK32_9 = MASKS32[9]; + private static final long MASK32_10 = MASKS32[10]; + private static final long MASK32_11 = MASKS32[11]; + private static final long MASK32_12 = MASKS32[12]; + private static final long MASK32_13 = MASKS32[13]; + private static final long MASK32_14 = MASKS32[14]; + private static final long MASK32_15 = MASKS32[15]; + private static final long MASK32_17 = MASKS32[17]; + private static final long MASK32_18 = MASKS32[18]; + private static final long MASK32_19 = MASKS32[19]; + private static final long MASK32_20 = MASKS32[20]; + private static final long MASK32_21 = MASKS32[21]; + private static final long MASK32_22 = MASKS32[22]; + private static final long MASK32_23 = MASKS32[23]; + private static final long MASK32_24 = MASKS32[24]; + + /** Decode 128 integers into {@code longs}. */ + void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + expand8(longs); + break; + case 2: + decode2(in, tmp, longs); + expand8(longs); + break; + case 3: + decode3(in, tmp, longs); + expand8(longs); + break; + case 4: + decode4(in, tmp, longs); + expand8(longs); + break; + case 5: + decode5(in, tmp, longs); + expand8(longs); + break; + case 6: + decode6(in, tmp, longs); + expand8(longs); + break; + case 7: + decode7(in, tmp, longs); + expand8(longs); + break; + case 8: + decode8(in, tmp, longs); + expand8(longs); + break; + case 9: + decode9(in, tmp, longs); + expand16(longs); + break; + case 10: + decode10(in, tmp, longs); + expand16(longs); + break; + case 11: + decode11(in, tmp, longs); + expand16(longs); + break; + case 12: + decode12(in, tmp, longs); + expand16(longs); + break; + case 13: + decode13(in, tmp, longs); + expand16(longs); + break; + case 14: + decode14(in, tmp, longs); + expand16(longs); + break; + case 15: + decode15(in, tmp, longs); + expand16(longs); + break; + case 16: + decode16(in, tmp, longs); + expand16(longs); + break; + case 17: + decode17(in, tmp, longs); + expand32(longs); + break; + case 18: + decode18(in, tmp, longs); + expand32(longs); + break; + case 19: + decode19(in, tmp, longs); + expand32(longs); + break; + case 20: + decode20(in, tmp, longs); + expand32(longs); + break; + case 21: + decode21(in, tmp, longs); + expand32(longs); + break; + case 22: + decode22(in, tmp, longs); + expand32(longs); + break; + case 23: + decode23(in, tmp, longs); + expand32(longs); + break; + case 24: + decode24(in, tmp, longs); + expand32(longs); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + expand32(longs); + break; + } + } + + /** Delta-decode 128 integers into {@code longs}. */ + void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) + throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + prefixSum8(longs, base); + break; + case 2: + decode2(in, tmp, longs); + prefixSum8(longs, base); + break; + case 3: + decode3(in, tmp, longs); + prefixSum8(longs, base); + break; + case 4: + decode4(in, tmp, longs); + prefixSum8(longs, base); + break; + case 5: + decode5(in, tmp, longs); + prefixSum8(longs, base); + break; + case 6: + decode6(in, tmp, longs); + prefixSum8(longs, base); + break; + case 7: + decode7(in, tmp, longs); + prefixSum8(longs, base); + break; + case 8: + decode8(in, tmp, longs); + prefixSum8(longs, base); + break; + case 9: + decode9(in, tmp, longs); + prefixSum16(longs, base); + break; + case 10: + decode10(in, tmp, longs); + prefixSum16(longs, base); + break; + case 11: + decode11(in, tmp, longs); + prefixSum16(longs, base); + break; + case 12: + decode12(in, tmp, longs); + prefixSum16(longs, base); + break; + case 13: + decode13(in, tmp, longs); + prefixSum16(longs, base); + break; + case 14: + decode14(in, tmp, longs); + prefixSum16(longs, base); + break; + case 15: + decode15(in, tmp, longs); + prefixSum16(longs, base); + break; + case 16: + decode16(in, tmp, longs); + prefixSum16(longs, base); + break; + case 17: + decode17(in, tmp, longs); + prefixSum32(longs, base); + break; + case 18: + decode18(in, tmp, longs); + prefixSum32(longs, base); + break; + case 19: + decode19(in, tmp, longs); + prefixSum32(longs, base); + break; + case 20: + decode20(in, tmp, longs); + prefixSum32(longs, base); + break; + case 21: + decode21(in, tmp, longs); + prefixSum32(longs, base); + break; + case 22: + decode22(in, tmp, longs); + prefixSum32(longs, base); + break; + case 23: + decode23(in, tmp, longs); + prefixSum32(longs, base); + break; + case 24: + decode24(in, tmp, longs); + prefixSum32(longs, base); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + prefixSum32(longs, base); + break; + } + } + + private static void decode1(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 2); + shiftLongs(tmp, 2, longs, 0, 7, MASK8_1); + shiftLongs(tmp, 2, longs, 2, 6, MASK8_1); + shiftLongs(tmp, 2, longs, 4, 5, MASK8_1); + shiftLongs(tmp, 2, longs, 6, 4, MASK8_1); + shiftLongs(tmp, 2, longs, 8, 3, MASK8_1); + shiftLongs(tmp, 2, longs, 10, 2, MASK8_1); + shiftLongs(tmp, 2, longs, 12, 1, MASK8_1); + shiftLongs(tmp, 2, longs, 14, 0, MASK8_1); + } + + private static void decode2(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 4); + shiftLongs(tmp, 4, longs, 0, 6, MASK8_2); + shiftLongs(tmp, 4, longs, 4, 4, MASK8_2); + shiftLongs(tmp, 4, longs, 8, 2, MASK8_2); + shiftLongs(tmp, 4, longs, 12, 0, MASK8_2); + } + + private static void decode3(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 6); + shiftLongs(tmp, 6, longs, 0, 5, MASK8_3); + shiftLongs(tmp, 6, longs, 6, 2, MASK8_3); + for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 2; ++iter, tmpIdx += 3, longsIdx += 2) { + long l0 = (tmp[tmpIdx + 0] & MASK8_2) << 1; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 2; + l1 |= (tmp[tmpIdx + 2] & MASK8_2) << 0; + longs[longsIdx + 1] = l1; + } + } + + private static void decode4(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 8); + shiftLongs(tmp, 8, longs, 0, 4, MASK8_4); + shiftLongs(tmp, 8, longs, 8, 0, MASK8_4); + } + + private static void decode5(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 10); + shiftLongs(tmp, 10, longs, 0, 3, MASK8_5); + for (int iter = 0, tmpIdx = 0, longsIdx = 10; iter < 2; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK8_3) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 4; + l1 |= (tmp[tmpIdx + 2] & MASK8_3) << 1; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK8_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK8_2) << 3; + l2 |= (tmp[tmpIdx + 4] & MASK8_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode6(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 12); + shiftLongs(tmp, 12, longs, 0, 2, MASK8_6); + shiftLongs(tmp, 12, tmp, 0, 0, MASK8_2); + for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 4; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 4; + l0 |= tmp[tmpIdx + 1] << 2; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode7(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 14); + shiftLongs(tmp, 14, longs, 0, 1, MASK8_7); + shiftLongs(tmp, 14, tmp, 0, 0, MASK8_1); + for (int iter = 0, tmpIdx = 0, longsIdx = 14; iter < 2; ++iter, tmpIdx += 7, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 6; + l0 |= tmp[tmpIdx + 1] << 5; + l0 |= tmp[tmpIdx + 2] << 4; + l0 |= tmp[tmpIdx + 3] << 3; + l0 |= tmp[tmpIdx + 4] << 2; + l0 |= tmp[tmpIdx + 5] << 1; + l0 |= tmp[tmpIdx + 6] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode8(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(longs, 0, 16); + } + + private static void decode9(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 18); + shiftLongs(tmp, 18, longs, 0, 7, MASK16_9); + for (int iter = 0, tmpIdx = 0, longsIdx = 18; iter < 2; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK16_7) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_5) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 3) & MASK16_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK16_1) << 8; + l3 |= (tmp[tmpIdx + 4] & MASK16_7) << 1; + l3 |= (tmp[tmpIdx + 5] >>> 6) & MASK16_1; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK16_6) << 3; + l4 |= (tmp[tmpIdx + 6] >>> 4) & MASK16_3; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK16_4) << 5; + l5 |= (tmp[tmpIdx + 7] >>> 2) & MASK16_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK16_2) << 7; + l6 |= (tmp[tmpIdx + 8] & MASK16_7) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode10(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 20); + shiftLongs(tmp, 20, longs, 0, 6, MASK16_10); + for (int iter = 0, tmpIdx = 0, longsIdx = 20; iter < 4; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_6) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_2) << 8; + l1 |= (tmp[tmpIdx + 2] & MASK16_6) << 2; + l1 |= (tmp[tmpIdx + 3] >>> 4) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK16_4) << 6; + l2 |= (tmp[tmpIdx + 4] & MASK16_6) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode11(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 22); + shiftLongs(tmp, 22, longs, 0, 5, MASK16_11); + for (int iter = 0, tmpIdx = 0, longsIdx = 22; iter < 2; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK16_5) << 6; + l0 |= (tmp[tmpIdx + 1] & MASK16_5) << 1; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK16_4) << 7; + l1 |= (tmp[tmpIdx + 3] & MASK16_5) << 2; + l1 |= (tmp[tmpIdx + 4] >>> 3) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK16_3) << 8; + l2 |= (tmp[tmpIdx + 5] & MASK16_5) << 3; + l2 |= (tmp[tmpIdx + 6] >>> 2) & MASK16_3; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK16_2) << 9; + l3 |= (tmp[tmpIdx + 7] & MASK16_5) << 4; + l3 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_4; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK16_1) << 10; + l4 |= (tmp[tmpIdx + 9] & MASK16_5) << 5; + l4 |= (tmp[tmpIdx + 10] & MASK16_5) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode12(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 24); + shiftLongs(tmp, 24, longs, 0, 4, MASK16_12); + shiftLongs(tmp, 24, tmp, 0, 0, MASK16_4); + for (int iter = 0, tmpIdx = 0, longsIdx = 24; iter < 8; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 8; + l0 |= tmp[tmpIdx + 1] << 4; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode13(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 26); + shiftLongs(tmp, 26, longs, 0, 3, MASK16_13); + for (int iter = 0, tmpIdx = 0, longsIdx = 26; iter < 2; ++iter, tmpIdx += 13, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_3) << 10; + l0 |= (tmp[tmpIdx + 1] & MASK16_3) << 7; + l0 |= (tmp[tmpIdx + 2] & MASK16_3) << 4; + l0 |= (tmp[tmpIdx + 3] & MASK16_3) << 1; + l0 |= (tmp[tmpIdx + 4] >>> 2) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 4] & MASK16_2) << 11; + l1 |= (tmp[tmpIdx + 5] & MASK16_3) << 8; + l1 |= (tmp[tmpIdx + 6] & MASK16_3) << 5; + l1 |= (tmp[tmpIdx + 7] & MASK16_3) << 2; + l1 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 8] & MASK16_1) << 12; + l2 |= (tmp[tmpIdx + 9] & MASK16_3) << 9; + l2 |= (tmp[tmpIdx + 10] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 11] & MASK16_3) << 3; + l2 |= (tmp[tmpIdx + 12] & MASK16_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode14(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 28); + shiftLongs(tmp, 28, longs, 0, 2, MASK16_14); + shiftLongs(tmp, 28, tmp, 0, 0, MASK16_2); + for (int iter = 0, tmpIdx = 0, longsIdx = 28; iter < 4; ++iter, tmpIdx += 7, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 12; + l0 |= tmp[tmpIdx + 1] << 10; + l0 |= tmp[tmpIdx + 2] << 8; + l0 |= tmp[tmpIdx + 3] << 6; + l0 |= tmp[tmpIdx + 4] << 4; + l0 |= tmp[tmpIdx + 5] << 2; + l0 |= tmp[tmpIdx + 6] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode15(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 30); + shiftLongs(tmp, 30, longs, 0, 1, MASK16_15); + shiftLongs(tmp, 30, tmp, 0, 0, MASK16_1); + for (int iter = 0, tmpIdx = 0, longsIdx = 30; iter < 2; ++iter, tmpIdx += 15, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 14; + l0 |= tmp[tmpIdx + 1] << 13; + l0 |= tmp[tmpIdx + 2] << 12; + l0 |= tmp[tmpIdx + 3] << 11; + l0 |= tmp[tmpIdx + 4] << 10; + l0 |= tmp[tmpIdx + 5] << 9; + l0 |= tmp[tmpIdx + 6] << 8; + l0 |= tmp[tmpIdx + 7] << 7; + l0 |= tmp[tmpIdx + 8] << 6; + l0 |= tmp[tmpIdx + 9] << 5; + l0 |= tmp[tmpIdx + 10] << 4; + l0 |= tmp[tmpIdx + 11] << 3; + l0 |= tmp[tmpIdx + 12] << 2; + l0 |= tmp[tmpIdx + 13] << 1; + l0 |= tmp[tmpIdx + 14] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode16(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(longs, 0, 32); + } + + private static void decode17(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 34); + shiftLongs(tmp, 34, longs, 0, 15, MASK32_17); + for (int iter = 0, tmpIdx = 0, longsIdx = 34; iter < 2; ++iter, tmpIdx += 17, longsIdx += 15) { + long l0 = (tmp[tmpIdx + 0] & MASK32_15) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_13) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 11) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_11) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 9) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_9) << 8; + l3 |= (tmp[tmpIdx + 4] >>> 7) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 4] & MASK32_7) << 10; + l4 |= (tmp[tmpIdx + 5] >>> 5) & MASK32_10; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 5] & MASK32_5) << 12; + l5 |= (tmp[tmpIdx + 6] >>> 3) & MASK32_12; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 6] & MASK32_3) << 14; + l6 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_14; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 7] & MASK32_1) << 16; + l7 |= (tmp[tmpIdx + 8] & MASK32_15) << 1; + l7 |= (tmp[tmpIdx + 9] >>> 14) & MASK32_1; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 9] & MASK32_14) << 3; + l8 |= (tmp[tmpIdx + 10] >>> 12) & MASK32_3; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 10] & MASK32_12) << 5; + l9 |= (tmp[tmpIdx + 11] >>> 10) & MASK32_5; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 11] & MASK32_10) << 7; + l10 |= (tmp[tmpIdx + 12] >>> 8) & MASK32_7; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 12] & MASK32_8) << 9; + l11 |= (tmp[tmpIdx + 13] >>> 6) & MASK32_9; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 13] & MASK32_6) << 11; + l12 |= (tmp[tmpIdx + 14] >>> 4) & MASK32_11; + longs[longsIdx + 12] = l12; + long l13 = (tmp[tmpIdx + 14] & MASK32_4) << 13; + l13 |= (tmp[tmpIdx + 15] >>> 2) & MASK32_13; + longs[longsIdx + 13] = l13; + long l14 = (tmp[tmpIdx + 15] & MASK32_2) << 15; + l14 |= (tmp[tmpIdx + 16] & MASK32_15) << 0; + longs[longsIdx + 14] = l14; + } + } + + private static void decode18(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 36); + shiftLongs(tmp, 36, longs, 0, 14, MASK32_18); + for (int iter = 0, tmpIdx = 0, longsIdx = 36; iter < 4; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK32_14) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_10) << 8; + l1 |= (tmp[tmpIdx + 2] >>> 6) & MASK32_8; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_6) << 12; + l2 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_12; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_2) << 16; + l3 |= (tmp[tmpIdx + 4] & MASK32_14) << 2; + l3 |= (tmp[tmpIdx + 5] >>> 12) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_12) << 6; + l4 |= (tmp[tmpIdx + 6] >>> 8) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK32_8) << 10; + l5 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK32_4) << 14; + l6 |= (tmp[tmpIdx + 8] & MASK32_14) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode19(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 38); + shiftLongs(tmp, 38, longs, 0, 13, MASK32_19); + for (int iter = 0, tmpIdx = 0, longsIdx = 38; iter < 2; ++iter, tmpIdx += 19, longsIdx += 13) { + long l0 = (tmp[tmpIdx + 0] & MASK32_13) << 6; + l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_7) << 12; + l1 |= (tmp[tmpIdx + 2] >>> 1) & MASK32_12; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_1) << 18; + l2 |= (tmp[tmpIdx + 3] & MASK32_13) << 5; + l2 |= (tmp[tmpIdx + 4] >>> 8) & MASK32_5; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 4] & MASK32_8) << 11; + l3 |= (tmp[tmpIdx + 5] >>> 2) & MASK32_11; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_2) << 17; + l4 |= (tmp[tmpIdx + 6] & MASK32_13) << 4; + l4 |= (tmp[tmpIdx + 7] >>> 9) & MASK32_4; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 7] & MASK32_9) << 10; + l5 |= (tmp[tmpIdx + 8] >>> 3) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 8] & MASK32_3) << 16; + l6 |= (tmp[tmpIdx + 9] & MASK32_13) << 3; + l6 |= (tmp[tmpIdx + 10] >>> 10) & MASK32_3; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 10] & MASK32_10) << 9; + l7 |= (tmp[tmpIdx + 11] >>> 4) & MASK32_9; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 11] & MASK32_4) << 15; + l8 |= (tmp[tmpIdx + 12] & MASK32_13) << 2; + l8 |= (tmp[tmpIdx + 13] >>> 11) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 13] & MASK32_11) << 8; + l9 |= (tmp[tmpIdx + 14] >>> 5) & MASK32_8; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 14] & MASK32_5) << 14; + l10 |= (tmp[tmpIdx + 15] & MASK32_13) << 1; + l10 |= (tmp[tmpIdx + 16] >>> 12) & MASK32_1; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 16] & MASK32_12) << 7; + l11 |= (tmp[tmpIdx + 17] >>> 6) & MASK32_7; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 17] & MASK32_6) << 13; + l12 |= (tmp[tmpIdx + 18] & MASK32_13) << 0; + longs[longsIdx + 12] = l12; + } + } + + private static void decode20(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 40); + shiftLongs(tmp, 40, longs, 0, 12, MASK32_20); + for (int iter = 0, tmpIdx = 0, longsIdx = 40; iter < 8; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK32_12) << 8; + l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_4) << 16; + l1 |= (tmp[tmpIdx + 2] & MASK32_12) << 4; + l1 |= (tmp[tmpIdx + 3] >>> 8) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_8) << 12; + l2 |= (tmp[tmpIdx + 4] & MASK32_12) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode21(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 42); + shiftLongs(tmp, 42, longs, 0, 11, MASK32_21); + for (int iter = 0, tmpIdx = 0, longsIdx = 42; iter < 2; ++iter, tmpIdx += 21, longsIdx += 11) { + long l0 = (tmp[tmpIdx + 0] & MASK32_11) << 10; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_1) << 20; + l1 |= (tmp[tmpIdx + 2] & MASK32_11) << 9; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_9; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_2) << 19; + l2 |= (tmp[tmpIdx + 4] & MASK32_11) << 8; + l2 |= (tmp[tmpIdx + 5] >>> 3) & MASK32_8; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 5] & MASK32_3) << 18; + l3 |= (tmp[tmpIdx + 6] & MASK32_11) << 7; + l3 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_7; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 7] & MASK32_4) << 17; + l4 |= (tmp[tmpIdx + 8] & MASK32_11) << 6; + l4 |= (tmp[tmpIdx + 9] >>> 5) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 9] & MASK32_5) << 16; + l5 |= (tmp[tmpIdx + 10] & MASK32_11) << 5; + l5 |= (tmp[tmpIdx + 11] >>> 6) & MASK32_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 11] & MASK32_6) << 15; + l6 |= (tmp[tmpIdx + 12] & MASK32_11) << 4; + l6 |= (tmp[tmpIdx + 13] >>> 7) & MASK32_4; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 13] & MASK32_7) << 14; + l7 |= (tmp[tmpIdx + 14] & MASK32_11) << 3; + l7 |= (tmp[tmpIdx + 15] >>> 8) & MASK32_3; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 15] & MASK32_8) << 13; + l8 |= (tmp[tmpIdx + 16] & MASK32_11) << 2; + l8 |= (tmp[tmpIdx + 17] >>> 9) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 17] & MASK32_9) << 12; + l9 |= (tmp[tmpIdx + 18] & MASK32_11) << 1; + l9 |= (tmp[tmpIdx + 19] >>> 10) & MASK32_1; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 19] & MASK32_10) << 11; + l10 |= (tmp[tmpIdx + 20] & MASK32_11) << 0; + longs[longsIdx + 10] = l10; + } + } + + private static void decode22(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 44); + shiftLongs(tmp, 44, longs, 0, 10, MASK32_22); + for (int iter = 0, tmpIdx = 0, longsIdx = 44; iter < 4; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK32_10) << 12; + l0 |= (tmp[tmpIdx + 1] & MASK32_10) << 2; + l0 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_8) << 14; + l1 |= (tmp[tmpIdx + 3] & MASK32_10) << 4; + l1 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK32_6) << 16; + l2 |= (tmp[tmpIdx + 5] & MASK32_10) << 6; + l2 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK32_4) << 18; + l3 |= (tmp[tmpIdx + 7] & MASK32_10) << 8; + l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK32_2) << 20; + l4 |= (tmp[tmpIdx + 9] & MASK32_10) << 10; + l4 |= (tmp[tmpIdx + 10] & MASK32_10) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode23(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 46); + shiftLongs(tmp, 46, longs, 0, 9, MASK32_23); + for (int iter = 0, tmpIdx = 0, longsIdx = 46; iter < 2; ++iter, tmpIdx += 23, longsIdx += 9) { + long l0 = (tmp[tmpIdx + 0] & MASK32_9) << 14; + l0 |= (tmp[tmpIdx + 1] & MASK32_9) << 5; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK32_5; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_4) << 19; + l1 |= (tmp[tmpIdx + 3] & MASK32_9) << 10; + l1 |= (tmp[tmpIdx + 4] & MASK32_9) << 1; + l1 |= (tmp[tmpIdx + 5] >>> 8) & MASK32_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 5] & MASK32_8) << 15; + l2 |= (tmp[tmpIdx + 6] & MASK32_9) << 6; + l2 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 7] & MASK32_3) << 20; + l3 |= (tmp[tmpIdx + 8] & MASK32_9) << 11; + l3 |= (tmp[tmpIdx + 9] & MASK32_9) << 2; + l3 |= (tmp[tmpIdx + 10] >>> 7) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 10] & MASK32_7) << 16; + l4 |= (tmp[tmpIdx + 11] & MASK32_9) << 7; + l4 |= (tmp[tmpIdx + 12] >>> 2) & MASK32_7; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 12] & MASK32_2) << 21; + l5 |= (tmp[tmpIdx + 13] & MASK32_9) << 12; + l5 |= (tmp[tmpIdx + 14] & MASK32_9) << 3; + l5 |= (tmp[tmpIdx + 15] >>> 6) & MASK32_3; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 15] & MASK32_6) << 17; + l6 |= (tmp[tmpIdx + 16] & MASK32_9) << 8; + l6 |= (tmp[tmpIdx + 17] >>> 1) & MASK32_8; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 17] & MASK32_1) << 22; + l7 |= (tmp[tmpIdx + 18] & MASK32_9) << 13; + l7 |= (tmp[tmpIdx + 19] & MASK32_9) << 4; + l7 |= (tmp[tmpIdx + 20] >>> 5) & MASK32_4; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 20] & MASK32_5) << 18; + l8 |= (tmp[tmpIdx + 21] & MASK32_9) << 9; + l8 |= (tmp[tmpIdx + 22] & MASK32_9) << 0; + longs[longsIdx + 8] = l8; + } + } + + private static void decode24(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 48); + shiftLongs(tmp, 48, longs, 0, 8, MASK32_24); + shiftLongs(tmp, 48, tmp, 0, 0, MASK32_8); + for (int iter = 0, tmpIdx = 0, longsIdx = 48; iter < 16; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 16; + l0 |= tmp[tmpIdx + 1] << 8; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java similarity index 82% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java index ce6c40f9402..d540abc85c2 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java @@ -14,12 +14,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.util.Objects; -import org.apache.lucene.codecs.*; -import org.apache.lucene.codecs.lucene90.*; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PointsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; +import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; +import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; +import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -98,7 +119,7 @@ public class Lucene99Codec extends Codec { super("Lucene99"); this.storedFieldsFormat = new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); - this.defaultPostingsFormat = new Lucene99PostingsFormat(); + this.defaultPostingsFormat = new Lucene912PostingsFormat(); this.defaultDVFormat = new Lucene90DocValuesFormat(); this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java index 877746641b4..7ff614d684e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; @@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.MultiLevelSkipListWriter; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; import org.apache.lucene.index.IndexOptions; @@ -339,7 +338,7 @@ import org.apache.lucene.util.packed.PackedInts; * * @lucene.experimental */ -public final class Lucene99PostingsFormat extends PostingsFormat { +public class Lucene99PostingsFormat extends PostingsFormat { /** * Filename extension for document number, frequencies, and skip data. See chapter:
impacts = acc.getCompetitiveFreqNormPairs(); diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java new file mode 100644 index 00000000000..9eddd3a0c55 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java @@ -0,0 +1,134 @@ +/* + * 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.backward_codecs.lucene99; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.LongHeap; +import org.apache.lucene.util.packed.PackedInts; + +/** Utility class to encode sequences of 128 small positive integers. */ +final class PForUtil { + + private static final int MAX_EXCEPTIONS = 7; + + static boolean allEqual(long[] l) { + for (int i = 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (l[i] != l[0]) { + return false; + } + } + return true; + } + + private final ForUtil forUtil; + + PForUtil(ForUtil forUtil) { + assert ForUtil.BLOCK_SIZE <= 256 : "blocksize must fit in one byte. got " + ForUtil.BLOCK_SIZE; + this.forUtil = forUtil; + } + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, DataOutput out) throws IOException { + // Determine the top MAX_EXCEPTIONS + 1 values + final LongHeap top = new LongHeap(MAX_EXCEPTIONS + 1); + for (int i = 0; i <= MAX_EXCEPTIONS; ++i) { + top.push(longs[i]); + } + long topValue = top.top(); + for (int i = MAX_EXCEPTIONS + 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (longs[i] > topValue) { + topValue = top.updateTop(longs[i]); + } + } + + long max = 0L; + for (int i = 1; i <= top.size(); ++i) { + max = Math.max(max, top.get(i)); + } + + final int maxBitsRequired = PackedInts.bitsRequired(max); + // We store the patch on a byte, so we can't decrease the number of bits required by more than 8 + final int patchedBitsRequired = + Math.max(PackedInts.bitsRequired(topValue), maxBitsRequired - 8); + int numExceptions = 0; + final long maxUnpatchedValue = (1L << patchedBitsRequired) - 1; + for (int i = 2; i <= top.size(); ++i) { + if (top.get(i) > maxUnpatchedValue) { + numExceptions++; + } + } + final byte[] exceptions = new byte[numExceptions * 2]; + if (numExceptions > 0) { + int exceptionCount = 0; + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + if (longs[i] > maxUnpatchedValue) { + exceptions[exceptionCount * 2] = (byte) i; + exceptions[exceptionCount * 2 + 1] = (byte) (longs[i] >>> patchedBitsRequired); + longs[i] &= maxUnpatchedValue; + exceptionCount++; + } + } + assert exceptionCount == numExceptions : exceptionCount + " " + numExceptions; + } + + if (allEqual(longs) && maxBitsRequired <= 8) { + for (int i = 0; i < numExceptions; ++i) { + exceptions[2 * i + 1] = + (byte) (Byte.toUnsignedLong(exceptions[2 * i + 1]) << patchedBitsRequired); + } + out.writeByte((byte) (numExceptions << 5)); + out.writeVLong(longs[0]); + } else { + final int token = (numExceptions << 5) | patchedBitsRequired; + out.writeByte((byte) token); + forUtil.encode(longs, patchedBitsRequired, out); + } + out.writeBytes(exceptions, exceptions.length); + } + + /** Decode 128 integers into {@code ints}. */ + void decode(DataInput in, long[] longs) throws IOException { + final int token = Byte.toUnsignedInt(in.readByte()); + final int bitsPerValue = token & 0x1f; + final int numExceptions = token >>> 5; + if (bitsPerValue == 0) { + Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong()); + } else { + forUtil.decode(bitsPerValue, in, longs); + } + for (int i = 0; i < numExceptions; ++i) { + longs[Byte.toUnsignedInt(in.readByte())] |= + Byte.toUnsignedLong(in.readByte()) << bitsPerValue; + } + } + + /** Skip 128 integers. */ + void skip(DataInput in) throws IOException { + final int token = Byte.toUnsignedInt(in.readByte()); + final int bitsPerValue = token & 0x1f; + final int numExceptions = token >>> 5; + if (bitsPerValue == 0) { + in.readVLong(); + in.skipBytes((numExceptions << 1)); + } else { + in.skipBytes(forUtil.numBytes(bitsPerValue) + (numExceptions << 1)); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PostingsUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PostingsUtil.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/PostingsUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PostingsUtil.java index 678754047b6..7b95bada5bc 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PostingsUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PostingsUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.IndexInput; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py new file mode 100644 index 00000000000..97a3a49a4ff --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py @@ -0,0 +1,524 @@ +#! /usr/bin/env python + +# 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. + +from math import gcd + +"""Code generation for ForUtil.java""" + +MAX_SPECIALIZED_BITS_PER_VALUE = 24 +OUTPUT_FILE = "ForUtil.java" +PRIMITIVE_SIZE = [8, 16, 32] +HEADER = """// This file has been automatically generated, DO NOT EDIT + +/* + * 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.backward_codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; + +// Inspired from https://fulmicoton.com/posts/bitpacking/ +// Encodes multiple integers in a long to get SIMD-like speedups. +// If bitsPerValue <= 8 then we pack 8 ints per long +// else if bitsPerValue <= 16 we pack 4 ints per long +// else we pack 2 ints per long +final class ForUtil { + + static final int BLOCK_SIZE = 128; + private static final int BLOCK_SIZE_LOG2 = 7; + + private static long expandMask32(long mask32) { + return mask32 | (mask32 << 32); + } + + private static long expandMask16(long mask16) { + return expandMask32(mask16 | (mask16 << 16)); + } + + private static long expandMask8(long mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + private static long mask32(int bitsPerValue) { + return expandMask32((1L << bitsPerValue) - 1); + } + + private static long mask16(int bitsPerValue) { + return expandMask16((1L << bitsPerValue) - 1); + } + + private static long mask8(int bitsPerValue) { + return expandMask8((1L << bitsPerValue) - 1); + } + + private static void expand8(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 56) & 0xFFL; + arr[16 + i] = (l >>> 48) & 0xFFL; + arr[32 + i] = (l >>> 40) & 0xFFL; + arr[48 + i] = (l >>> 32) & 0xFFL; + arr[64 + i] = (l >>> 24) & 0xFFL; + arr[80 + i] = (l >>> 16) & 0xFFL; + arr[96 + i] = (l >>> 8) & 0xFFL; + arr[112 + i] = l & 0xFFL; + } + } + + private static void expand8To32(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 24) & 0x000000FF000000FFL; + arr[16 + i] = (l >>> 16) & 0x000000FF000000FFL; + arr[32 + i] = (l >>> 8) & 0x000000FF000000FFL; + arr[48 + i] = l & 0x000000FF000000FFL; + } + } + + private static void collapse8(long[] arr) { + for (int i = 0; i < 16; ++i) { + arr[i] = + (arr[i] << 56) + | (arr[16 + i] << 48) + | (arr[32 + i] << 40) + | (arr[48 + i] << 32) + | (arr[64 + i] << 24) + | (arr[80 + i] << 16) + | (arr[96 + i] << 8) + | arr[112 + i]; + } + } + + private static void expand16(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 48) & 0xFFFFL; + arr[32 + i] = (l >>> 32) & 0xFFFFL; + arr[64 + i] = (l >>> 16) & 0xFFFFL; + arr[96 + i] = l & 0xFFFFL; + } + } + + private static void expand16To32(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL; + arr[32 + i] = l & 0x0000FFFF0000FFFFL; + } + } + + private static void collapse16(long[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = (arr[i] << 48) | (arr[32 + i] << 32) | (arr[64 + i] << 16) | arr[96 + i]; + } + } + + private static void expand32(long[] arr) { + for (int i = 0; i < 64; ++i) { + long l = arr[i]; + arr[i] = l >>> 32; + arr[64 + i] = l & 0xFFFFFFFFL; + } + } + + private static void collapse32(long[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 32) | arr[64 + i]; + } + } + + private static void prefixSum8(long[] arr, long base) { + expand8To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum16(long[] arr, long base) { + // We need to move to the next primitive size to avoid overflows + expand16To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum32(long[] arr, long base) { + arr[0] += base << 32; + innerPrefixSum32(arr); + expand32(arr); + final long l = arr[BLOCK_SIZE/2-1]; + for (int i = BLOCK_SIZE/2; i < BLOCK_SIZE; ++i) { + arr[i] += l; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum32(long[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + arr[32] += arr[31]; + arr[33] += arr[32]; + arr[34] += arr[33]; + arr[35] += arr[34]; + arr[36] += arr[35]; + arr[37] += arr[36]; + arr[38] += arr[37]; + arr[39] += arr[38]; + arr[40] += arr[39]; + arr[41] += arr[40]; + arr[42] += arr[41]; + arr[43] += arr[42]; + arr[44] += arr[43]; + arr[45] += arr[44]; + arr[46] += arr[45]; + arr[47] += arr[46]; + arr[48] += arr[47]; + arr[49] += arr[48]; + arr[50] += arr[49]; + arr[51] += arr[50]; + arr[52] += arr[51]; + arr[53] += arr[52]; + arr[54] += arr[53]; + arr[55] += arr[54]; + arr[56] += arr[55]; + arr[57] += arr[56]; + arr[58] += arr[57]; + arr[59] += arr[58]; + arr[60] += arr[59]; + arr[61] += arr[60]; + arr[62] += arr[61]; + arr[63] += arr[62]; + } + + private final long[] tmp = new long[BLOCK_SIZE / 2]; + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + final int numLongs; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + numLongs = BLOCK_SIZE / 8; + collapse8(longs); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + numLongs = BLOCK_SIZE / 4; + collapse16(longs); + } else { + nextPrimitive = 32; + numLongs = BLOCK_SIZE / 2; + collapse32(longs); + } + + final int numLongsPerShift = bitsPerValue * 2; + int idx = 0; + int shift = nextPrimitive - bitsPerValue; + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] = longs[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] |= longs[idx++] << shift; + } + } + + final int remainingBitsPerLong = shift + bitsPerValue; + final long maskRemainingBitsPerLong; + if (nextPrimitive == 8) { + maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong]; + } else if (nextPrimitive == 16) { + maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong]; + } else { + maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numLongs) { + if (remainingBitsPerValue >= remainingBitsPerLong) { + remainingBitsPerValue -= remainingBitsPerLong; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final long mask1, mask2; + if (nextPrimitive == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue]; + } else if (nextPrimitive == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numLongsPerShift; ++i) { + out.writeLong(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + throws IOException { + final int numLongs = bitsPerValue << 1; + in.readLongs(tmp, 0, numLongs); + final long mask = MASKS32[bitsPerValue]; + int longsIdx = 0; + int shift = 32 - bitsPerValue; + for (; shift >= 0; shift -= bitsPerValue) { + shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); + longsIdx += numLongs; + } + final int remainingBitsPerLong = shift + bitsPerValue; + final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerLong; + for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) { + int b = bitsPerValue - remainingBits; + long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerLong) { + b -= remainingBitsPerLong; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerLong - b)) & MASKS32[b]; + remainingBits = remainingBitsPerLong - b; + } else { + remainingBits = remainingBitsPerLong; + } + longs[longsIdx] = l; + } + } + + /** + * The pattern that this shiftLongs method applies is recognized by the C2 compiler, which + * generates SIMD instructions for it in order to shift multiple longs at once. + */ + private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = (a[i] >>> shift) & mask; + } + } + +""" + +def writeRemainderWithSIMDOptimize(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): + iteration = 1 + num_longs = bpv * num_values / remaining_bits_per_long + while num_longs % 2 == 0 and num_values % 2 == 0: + num_longs /= 2 + num_values /= 2 + iteration *= 2 + + f.write(' shiftLongs(tmp, %d, tmp, 0, 0, MASK%d_%d);\n' % (iteration * num_longs, next_primitive, remaining_bits_per_long)) + f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values)) + tmp_idx = 0 + b = bpv + b -= remaining_bits_per_long + f.write(' long l0 = tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) + tmp_idx += 1 + while b >= remaining_bits_per_long: + b -= remaining_bits_per_long + f.write(' l0 |= tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) + tmp_idx += 1 + f.write(' longs[longsIdx + 0] = l0;\n') + f.write(' }\n') + + +def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): + iteration = 1 + num_longs = bpv * num_values / remaining_bits_per_long + while num_longs % 2 == 0 and num_values % 2 == 0: + num_longs /= 2 + num_values /= 2 + iteration *= 2 + f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values)) + i = 0 + remaining_bits = 0 + tmp_idx = 0 + for i in range(int(num_values)): + b = bpv + if remaining_bits == 0: + b -= remaining_bits_per_long + f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b)) + else: + b -= remaining_bits + f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b)) + tmp_idx += 1 + while b >= remaining_bits_per_long: + b -= remaining_bits_per_long + f.write(' l%d |= (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b)) + tmp_idx += 1 + if b > 0: + f.write(' l%d |= (tmp[tmpIdx + %d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_long-b, next_primitive, b)) + remaining_bits = remaining_bits_per_long-b + f.write(' longs[longsIdx + %d] = l%d;\n' %(i, i)) + f.write(' }\n') + + +def writeDecode(bpv, f): + next_primitive = 32 + if bpv <= 8: + next_primitive = 8 + elif bpv <= 16: + next_primitive = 16 + f.write(' private static void decode%d(DataInput in, long[] tmp, long[] longs) throws IOException {\n' %bpv) + num_values_per_long = 64 / next_primitive + if bpv == next_primitive: + f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2)) + else: + f.write(' in.readLongs(tmp, 0, %d);\n' %(bpv*2)) + shift = next_primitive - bpv + o = 0 + while shift >= 0: + f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv)) + o += bpv*2 + shift -= bpv + if shift + bpv > 0: + if bpv % (next_primitive % bpv) == 0: + writeRemainderWithSIMDOptimize(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) + else: + writeRemainder(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) + f.write(' }\n') + + +if __name__ == '__main__': + f = open(OUTPUT_FILE, 'w') + f.write(HEADER) + for primitive_size in PRIMITIVE_SIZE: + f.write(' private static final long[] MASKS%d = new long[%d];\n' %(primitive_size, primitive_size)) + f.write('\n') + f.write(' static {\n') + for primitive_size in PRIMITIVE_SIZE: + f.write(' for (int i = 0; i < %d; ++i) {\n' %primitive_size) + f.write(' MASKS%d[i] = mask%d(i);\n' %(primitive_size, primitive_size)) + f.write(' }\n') + f.write(' }') + f.write(""" + // mark values in array as final longs to avoid the cost of reading array, arrays should only be + // used when the idx is a variable +""") + for primitive_size in PRIMITIVE_SIZE: + for bpv in range(1, min(MAX_SPECIALIZED_BITS_PER_VALUE + 1, primitive_size)): + if bpv * 2 != primitive_size or primitive_size == 8: + f.write(' private static final long MASK%d_%d = MASKS%d[%d];\n' %(primitive_size, bpv, primitive_size, bpv)) + + f.write(""" + /** Decode 128 integers into {@code longs}. */ + void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + switch (bitsPerValue) { +""") + for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + next_primitive = 32 + if bpv <= 8: + next_primitive = 8 + elif bpv <= 16: + next_primitive = 16 + f.write(' case %d:\n' %bpv) + f.write(' decode%d(in, tmp, longs);\n' %bpv) + f.write(' expand%d(longs);\n' %next_primitive) + f.write(' break;\n') + f.write(' default:\n') + f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') + f.write(' expand32(longs);\n') + f.write(' break;\n') + f.write(' }\n') + f.write(' }\n') + + f.write(""" + /** + * Delta-decode 128 integers into {@code longs}. + */ + void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) throws IOException { + switch (bitsPerValue) { +""") + for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + next_primitive = 32 + if bpv <= 8: + next_primitive = 8 + elif bpv <= 16: + next_primitive = 16 + f.write(' case %d:\n' %bpv) + f.write(' decode%d(in, tmp, longs);\n' %bpv) + f.write(' prefixSum%d(longs, base);\n' %next_primitive) + f.write(' break;\n') + f.write(' default:\n') + f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') + f.write(' prefixSum32(longs, base);\n') + f.write(' break;\n') + f.write(' }\n') + f.write(' }\n') + + f.write('\n') + for i in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + writeDecode(i, f) + if i < MAX_SPECIALIZED_BITS_PER_VALUE: + f.write('\n') + + f.write('}\n') diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java new file mode 100644 index 00000000000..6c1b0b44a07 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java @@ -0,0 +1,428 @@ +/* + * 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. + */ + +/** + * Lucene 9.9 file format. + * + *

Apache Lucene - Index File Formats

+ * + * + * + *

Introduction

+ * + *
+ * + *

This document defines the index file formats used in this version of Lucene. If you are using + * a different version of Lucene, please consult the copy of docs/ that was distributed + * with the version you are using. + * + *

This document attempts to provide a high-level definition of the Apache Lucene file formats. + *

+ * + *

Definitions

+ * + *
+ * + *

The fundamental concepts in Lucene are index, document, field and term. + * + *

An index contains a sequence of documents. + * + *

    + *
  • A document is a sequence of fields. + *
  • A field is a named sequence of terms. + *
  • A term is a sequence of bytes. + *
+ * + *

The same sequence of bytes in two different fields is considered a different term. Thus terms + * are represented as a pair: the string naming the field, and the bytes within the field. + * + *

Inverted Indexing

+ * + *

Lucene's index stores terms and statistics about those terms in order to make term-based + * search more efficient. Lucene's terms index falls into the family of indexes known as an + * inverted index. This is because it can list, for a term, the documents that contain it. + * This is the inverse of the natural relationship, in which documents list terms. + * + *

Types of Fields

+ * + *

In Lucene, fields may be stored, in which case their text is stored in the index + * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field + * may be both stored and indexed. + * + *

The text of a field may be tokenized into terms to be indexed, or the text of a field + * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is + * useful for certain identifier fields to be indexed literally. + * + *

See the {@link org.apache.lucene.document.Field Field} java docs for more information on + * Fields. + * + *

Segments

+ * + *

Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a + * fully independent index, which could be searched separately. Indexes evolve by: + * + *

    + *
  1. Creating new segments for newly added documents. + *
  2. Merging existing segments. + *
+ * + *

Searches may involve multiple segments and/or multiple indexes, each index potentially + * composed of a set of segments. + * + *

Document Numbers

+ * + *

Internally, Lucene refers to documents by an integer document number. The first + * document added to an index is numbered zero, and each subsequent document added gets a number one + * greater than the previous. + * + *

Note that a document's number may change, so caution should be taken when storing these + * numbers outside of Lucene. In particular, numbers may change in the following situations: + * + *

    + *
  • + *

    The numbers stored in each segment are unique only within the segment, and must be + * converted before they can be used in a larger context. The standard technique is to + * allocate each segment a range of values, based on the range of numbers used in that + * segment. To convert a document number from a segment to an external value, the segment's + * base document number is added. To convert an external value back to a + * segment-specific value, the segment is identified by the range that the external value is + * in, and the segment's base value is subtracted. For example two five document segments + * might be combined, so that the first segment has a base value of zero, and the second of + * five. Document three from the second segment would have an external value of eight. + *

  • + *

    When documents are deleted, gaps are created in the numbering. These are eventually + * removed as the index evolves through merging. Deleted documents are dropped when segments + * are merged. A freshly-merged segment thus has no gaps in its numbering. + *

+ * + *
+ * + *

Index Structure Overview

+ * + *
+ * + *

Each segment index maintains the following: + * + *

    + *
  • {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This + * contains metadata about a segment, such as the number of documents, what files it uses, and + * information about how the segment is sorted + *
  • {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This + * contains metadata about the set of named fields used in the index. + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}. + * This contains, for each document, a list of attribute-value pairs, where the attributes are + * field names. These are used to store auxiliary information about the document, such as its + * title, url, or an identifier to access a database. The set of stored fields are what is + * returned for each hit when searching. This is keyed by document number. + *
  • {@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term dictionary}. + * A dictionary containing all of the terms used in all of the indexed fields of all of the + * documents. The dictionary also contains the number of documents which contain the term, and + * pointers to the term's frequency and proximity data. + *
  • {@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Frequency + * data}. For each term in the dictionary, the numbers of all the documents that contain that + * term, and the frequency of the term in that document, unless frequencies are omitted + * ({@link org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) + *
  • {@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Proximity + * data}. For each term in the dictionary, the positions that the term occurs in each + * document. Note that this will not exist if all fields in all documents omit position data. + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For + * each field in each document, a value is stored that is multiplied into the score for hits + * on that field. + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each + * field in each document, the term vector (sometimes called document vector) may be stored. A + * term vector consists of term text and term frequency. To add Term Vectors to your index see + * the {@link org.apache.lucene.document.Field Field} constructors + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like + * stored values, these are also keyed by document number, but are generally intended to be + * loaded into main memory for fast access. Whereas stored values are generally intended for + * summary results from searches, per-document values are useful for things like scoring + * factors. + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An + * optional file indicating which documents are live. + *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair + * of files, recording dimensionally indexed fields, to enable fast numeric range filtering + * and large numeric values like BigInteger and BigDecimal (1D) and geographic shape + * intersection (2D, 3D). + *
  • {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The + * vector format stores numeric vectors in a format optimized for random access and + * computation, supporting high-dimensional nearest-neighbor search. + *
+ * + *

Details on each of these are provided in their linked pages.

+ * + *

File Naming

+ * + *
+ * + *

All files belonging to a segment have the same name with varying extensions. The extensions + * correspond to the different file formats described below. When using the Compound File format + * (default for small segments) these files (except for the Segment info file, the Lock file, and + * Deleted documents file) are collapsed into a single .cfs file (see below for details) + * + *

Typically, all segments in an index are stored in a single directory, although this is not + * required. + * + *

File names are never re-used. That is, when any file is saved to the Directory it is given a + * never before used filename. This is achieved using a simple generations approach. For example, + * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long + * integer represented in alpha-numeric (base 36) form.

+ * + *

Summary of File Extensions

+ * + *
+ * + *

The following table summarizes the names and extensions of the files in Lucene: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
lucene filenames by extension
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same + * file.
{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for + * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.dii, .dimHolds indexed points
{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}.vec, .vem, .veq, vexHolds indexed vectors; .vec files contain the raw vector data, + * .vem the vector metadata, .veq the quantized vector data, and .vex the + * hnsw graph data.
+ * + *

+ * + *

Lock File

+ * + * The write lock, which is stored in the index directory by default, is named "write.lock". If the + * lock directory is different from the index directory then the write lock will be named + * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index + * directory. When this file is present, a writer is currently modifying the index (adding or + * removing documents). This lock file ensures that only one writer is modifying the index at a + * time. + * + *

History

+ * + *

Compatibility notes are provided in this document, describing how file formats have changed + * from prior versions: + * + *

    + *
  • In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit + * lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching + * or adding/deleting of docs. When the new segments file is saved (committed), it will be + * written in the new file format (meaning no specific "upgrade" process is needed). But note + * that once a commit has occurred, pre-2.1 Lucene will not be able to read the index. + *
  • In version 2.3, the file format was changed to allow segments to share a single set of doc + * store (vectors & stored fields) files. This allows for faster indexing in certain + * cases. The change is fully backwards compatible (in the same way as the lock-less commits + * change in 2.1). + *
  • In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified + * UTF-8. See LUCENE-510 for + * details. + *
  • In version 2.9, an optional opaque Map<String,String> CommitUserData may be passed to + * IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N + * file. See LUCENE-1382 for + * details. Also, diagnostics were added to each segment written recording details about why + * it was written (due to flush, merge; which OS/JRE was used; etc.). See issue LUCENE-1654 for details. + *
  • In version 3.0, compressed fields are no longer written to the index (they can still be + * read, but on merge the new segment will write them, uncompressed). See issue LUCENE-1960 for details. + *
  • In version 3.1, segments records the code version that created them. See LUCENE-2720 for details. + * Additionally segments track explicitly whether or not they have term vectors. See LUCENE-2811 for details. + *
  • In version 3.2, numeric fields are written as natively to stored fields file, previously + * they were stored in text format only. + *
  • In version 3.4, fields can omit position data while still indexing term frequencies. + *
  • In version 4.0, the format of the inverted index became extensible via the {@link + * org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues}) + * was introduced. Normalization factors need no longer be a single byte, they can be any + * {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be + * unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into + * the postings lists. Payloads can be stored in the term vectors. + *
  • In version 4.1, the format of the postings list changed to use either of FOR compression or + * variable-byte encoding, depending upon the frequency of the term. Terms appearing only once + * were changed to inline directly into the term dictionary. Stored fields are compressed by + * default. + *
  • In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued + * type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields. + *
  • In version 4.5, DocValues were extended to explicitly represent missing values. + *
  • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to + * allow updating NumericDocValues fields. + *
  • In version 4.8, checksum footers were added to the end of each index file for improved data + * integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32 + * checksum of the file. + *
  • In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is + * suitable for faceting/sorting/analytics. + *
  • In version 5.4, DocValues have been improved to store more information on disk: addresses + * for binary fields and ord indexes for multi-valued fields. + *
  • In version 6.0, Points were added, for multi-dimensional range/distance search. + *
  • In version 6.2, new Segment info format that reads/writes the index sort, to support index + * sorting. + *
  • In version 7.0, DocValues have been improved to better support sparse doc values thanks to + * an iterator API. + *
  • In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term + * freq, normalization factor) pairs that may trigger the maximum score of the block. This + * information is recorded alongside skip data in order to be able to skip blocks of doc ids + * if they may not produce high enough scores. Additionally doc values and norms has been + * extended with jump-tables to make access O(1) instead of O(n), where n is the number of + * elements to skip when advancing in the data. + *
  • In version 8.4, postings, positions, offsets and payload lengths have move to a more + * performant encoding that is vectorized. + *
  • In version 8.6, index sort serialization is delegated to the sorts themselves, to allow + * user-defined sorts to be used + *
  • In version 8.7, stored fields compression became adaptive to better handle documents with + * smaller stored fields. + *
  • In version 9.0, vector-valued fields were added. + *
  • In version 9.1, vector-valued fields were modified to add a graph hierarchy. + *
  • In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by + * IndexDISI. ordToDoc mappings was added to .vem. + *
  • In version 9.5, HNSW graph connections were changed to be delta-encoded with vints. + * Additionally, metadata file size improvements were made by delta-encoding nodes by graph + * layer and not writing the node ids for the zeroth layer. + *
  • In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector + * format to utilize int8 quantized vectors for float32 vector search. + *
+ * + * + * + *

Limitations

+ * + *
+ * + *

Lucene uses a Java int to refer to document numbers, and the index file format + * uses an Int32 on-disk to store document numbers. This is a limitation of both the + * index file format and the current implementation. Eventually these should be replaced with either + * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt + * VInt} values which have no limit.

+ */ +package org.apache.lucene.backward_codecs.lucene99; diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index 36cb0bbbd24..df14387fc68 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -22,3 +22,4 @@ org.apache.lucene.backward_codecs.lucene91.Lucene91Codec org.apache.lucene.backward_codecs.lucene92.Lucene92Codec org.apache.lucene.backward_codecs.lucene94.Lucene94Codec org.apache.lucene.backward_codecs.lucene95.Lucene95Codec +org.apache.lucene.backward_codecs.lucene99.Lucene99Codec diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 18ac359d2ae..9733362abe7 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -16,3 +16,4 @@ org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat +org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java index 7965bc9c778..814285a8895 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java @@ -23,12 +23,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.lucene.backward_codecs.lucene90.Lucene90ScoreSkipReader.MutableImpactList; +import org.apache.lucene.backward_codecs.lucene99.Lucene99SkipWriter; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.lucene90.blocktree.FieldReader; import org.apache.lucene.codecs.lucene90.blocktree.Stats; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99SkipWriter; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.DirectoryReader; @@ -77,22 +76,6 @@ public class TestLucene90PostingsFormat extends BasePostingsFormatTestCase { d.close(); } - private void shouldFail(int minItemsInBlock, int maxItemsInBlock) { - expectThrows( - IllegalArgumentException.class, - () -> { - new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock); - }); - } - - public void testInvalidBlockSizes() throws Exception { - shouldFail(0, 0); - shouldFail(10, 8); - shouldFail(-1, 10); - shouldFail(10, -1); - shouldFail(10, 12); - } - public void testImpactSerialization() throws IOException { // omit norms and omit freqs doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java index 61949cfe227..44ff826e159 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java @@ -14,22 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; -import static org.apache.lucene.codecs.lucene99.ForUtil.BLOCK_SIZE; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.POS_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene99.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.POS_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.PushPostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexFileNames; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java new file mode 100644 index 00000000000..f513562cb35 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java @@ -0,0 +1,68 @@ +/* + * 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.backward_codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.util.IOUtils; + +public class Lucene99RWPostingsFormat extends Lucene99PostingsFormat { + + private final int minTermBlockSize; + private final int maxTermBlockSize; + + /** Creates {@code Lucene99PostingsFormat} with default settings. */ + public Lucene99RWPostingsFormat() { + this( + Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + /** + * Creates {@code Lucene99PostingsFormat} with custom values for {@code minBlockSize} and {@code + * maxBlockSize} passed to block terms dictionary. + * + * @see + * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) + */ + public Lucene99RWPostingsFormat(int minTermBlockSize, int maxTermBlockSize) { + super(); + Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); + this.minTermBlockSize = minTermBlockSize; + this.maxTermBlockSize = maxTermBlockSize; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + boolean success = false; + try { + FieldsConsumer ret = + new Lucene90BlockTreeTermsWriter( + state, postingsWriter, minTermBlockSize, maxTermBlockSize); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsWriter); + } + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForDeltaUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForDeltaUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java index 29448ff1e41..e21eaab461c 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForDeltaUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForUtil.java index 2f179c743af..85e0dcaebc4 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java index 0e0d63963cc..01d5b4ab17f 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java @@ -19,7 +19,6 @@ package org.apache.lucene.backward_codecs.lucene99; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java similarity index 90% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java index 341805e8a3e..e954af88c3b 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java @@ -14,22 +14,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; -import static org.apache.lucene.codecs.lucene99.Lucene99ScoreSkipReader.readImpacts; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.readImpacts; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.MutableImpactList; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.lucene90.blocktree.FieldReader; import org.apache.lucene.codecs.lucene90.blocktree.Stats; -import org.apache.lucene.codecs.lucene99.Lucene99ScoreSkipReader.MutableImpactList; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; -import org.apache.lucene.index.*; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -41,7 +45,7 @@ import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase { - private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99PostingsFormat()); + private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99RWPostingsFormat()); @Override protected Codec getCodec() { @@ -77,7 +81,7 @@ public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase { expectThrows( IllegalArgumentException.class, () -> { - new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock); + new Lucene99RWPostingsFormat(minItemsInBlock, maxItemsInBlock); }); } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPForUtil.java index 16f719e93b2..528d5d2b960 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPostingsUtil.java similarity index 97% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPostingsUtil.java index b9cb3f20dba..736bdca51be 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPostingsUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.Directory; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java index d840a00ca5b..8db406df992 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java @@ -20,9 +20,9 @@ import static org.apache.lucene.backward_index.TestBasicBackwardsCompatibility.a import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene99.Lucene99Codec; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.document.Document; diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java index d389372d8f8..db704148573 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java @@ -23,13 +23,13 @@ import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; -/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene99PostingsWriter}. */ +/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene912PostingsWriter}. */ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { private final int minTermBlockSize; @@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { @@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); boolean success = false; try { FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java index 89494297c2a..8e17edb9e2c 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java @@ -24,7 +24,7 @@ import java.util.TreeMap; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.Fields; @@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.TransitionAccessor; // - or: longer dense skip lists than just next byte? /** - * Wraps {@link Lucene99PostingsFormat} format for on-disk storage, but then at read time loads and + * Wraps {@link Lucene912PostingsFormat} format for on-disk storage, but then at read time loads and * stores all terms and postings directly in RAM as byte[], int[]. * *

WARNING: This is exceptionally RAM intensive: it makes no effort to compress the @@ -97,12 +97,12 @@ public final class DirectPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return PostingsFormat.forName("Lucene99").fieldsConsumer(state); + return PostingsFormat.forName("Lucene912").fieldsConsumer(state); } @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - FieldsProducer postings = PostingsFormat.forName("Lucene99").fieldsProducer(state); + FieldsProducer postings = PostingsFormat.forName("Lucene912").fieldsProducer(state); if (state.context.context() != IOContext.Context.MERGE) { FieldsProducer loadedPostings; try { diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java index 0a18b75ac00..b184f85b176 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java @@ -22,8 +22,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -41,7 +41,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { @@ -57,7 +57,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); boolean success = false; try { FieldsProducer ret = new FSTTermsReader(state, postingsReader); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java index ae01216ccf6..1f876e5e9d1 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java @@ -17,13 +17,13 @@ package org.apache.lucene.codecs.uniformsplit; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.TermState; @@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator; /** * {@link TermState} serializer which encodes each file pointer as a delta relative to a base file - * pointer. It differs from {@link Lucene99PostingsWriter#encodeTerm} which encodes each file + * pointer. It differs from {@link Lucene912PostingsWriter#encodeTerm} which encodes each file * pointer as a delta relative to the previous file pointer. * *

It automatically sets the base file pointer to the first valid file pointer for doc start FP, @@ -95,7 +95,7 @@ public class DeltaBaseTermStateSerializer implements Accountable { /** * Writes a {@link BlockTermState} to the provided {@link DataOutput}. * - *

Simpler variant of {@link Lucene99PostingsWriter#encodeTerm(DataOutput, FieldInfo, + *

Simpler variant of {@link Lucene912PostingsWriter#encodeTerm(DataOutput, FieldInfo, * BlockTermState, boolean)}. */ public void writeTermState( @@ -140,15 +140,12 @@ public class DeltaBaseTermStateSerializer implements Accountable { termStatesOutput.writeVLong(intTermState.lastPosBlockOffset); } } - if (intTermState.skipOffset != -1) { - termStatesOutput.writeVLong(intTermState.skipOffset); - } } /** * Reads a {@link BlockTermState} from the provided {@link DataInput}. * - *

Simpler variant of {@link Lucene99PostingsReader#decodeTerm(DataInput, FieldInfo, + *

Simpler variant of {@link Lucene912PostingsReader#decodeTerm(DataInput, FieldInfo, * BlockTermState, boolean)}. * * @param reuse {@link BlockTermState} to reuse; or null to create a new one. @@ -190,9 +187,6 @@ public class DeltaBaseTermStateSerializer implements Accountable { intTermState.lastPosBlockOffset = termStatesInput.readVLong(); } } - if (intTermState.docFreq > BLOCK_SIZE) { - intTermState.skipOffset = termStatesInput.readVLong(); - } return intTermState; } @@ -210,7 +204,6 @@ public class DeltaBaseTermStateSerializer implements Accountable { termState.docStartFP = 0; termState.posStartFP = 0; termState.payStartFP = 0; - termState.skipOffset = -1; termState.lastPosBlockOffset = -1; termState.singletonDocID = -1; diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java index 8e2b7bb7d54..c8a19bf9da9 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -113,7 +113,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { FieldsConsumer termsWriter = @@ -130,7 +130,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); boolean success = false; try { FieldsProducer termsReader = diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java index a0fa67508f3..dc77bc710a1 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java @@ -28,7 +28,7 @@ * org.apache.lucene.search.PhraseQuery}) *

  • Quite efficient for {@link org.apache.lucene.search.PrefixQuery} *
  • Not efficient for spell-check and {@link org.apache.lucene.search.FuzzyQuery}, in this case - * prefer {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat} + * prefer {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat} * */ package org.apache.lucene.codecs.uniformsplit; diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java index 8ce34b0e566..ab20ee67c8c 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.KnnByteVectorField; @@ -42,7 +42,7 @@ import org.apache.lucene.tests.index.BaseIndexFileFormatTestCase; public class TestHnswBitVectorsFormat extends BaseIndexFileFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new HnswBitVectorsFormat(); diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java index dde59993f6e..51891b9d0ad 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java @@ -17,7 +17,7 @@ package org.apache.lucene.codecs.lucene90.tests; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; /** Test utility class to create mock {@link IntBlockTermState}. */ public class MockTermStateFactory { diff --git a/lucene/core/src/generated/checksums/generateForUtil.json b/lucene/core/src/generated/checksums/generateForUtil.json index 527070d7bf0..752285f4d7f 100644 --- a/lucene/core/src/generated/checksums/generateForUtil.json +++ b/lucene/core/src/generated/checksums/generateForUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java": "1292ad354d255b1272ffd3db684aa2ddb2bc49ec", - "lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py": "ab7b63a1b73986cc04e43de1c8f474b97aef5116" + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "5ff856e80cab30f9e5704aa89f3197f017d07624", + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "3ccf92b3ddbff6340a13e8a55090bfb900dc7be2" } \ No newline at end of file diff --git a/lucene/core/src/java/module-info.java b/lucene/core/src/java/module-info.java index 9df3adf7aef..a0f0bad01eb 100644 --- a/lucene/core/src/java/module-info.java +++ b/lucene/core/src/java/module-info.java @@ -15,7 +15,7 @@ * limitations under the License. */ -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; /** Lucene Core. */ @SuppressWarnings("module") // the test framework is compiled after the core... @@ -33,6 +33,7 @@ module org.apache.lucene.core { exports org.apache.lucene.codecs.lucene94; exports org.apache.lucene.codecs.lucene95; exports org.apache.lucene.codecs.lucene99; + exports org.apache.lucene.codecs.lucene912; exports org.apache.lucene.codecs.perfield; exports org.apache.lucene.codecs; exports org.apache.lucene.document; @@ -71,7 +72,7 @@ module org.apache.lucene.core { provides org.apache.lucene.analysis.TokenizerFactory with org.apache.lucene.analysis.standard.StandardTokenizerFactory; provides org.apache.lucene.codecs.Codec with - Lucene99Codec; + Lucene912Codec; provides org.apache.lucene.codecs.DocValuesFormat with org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with @@ -79,7 +80,7 @@ module org.apache.lucene.core { org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat, org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat; provides org.apache.lucene.codecs.PostingsFormat with - org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; + org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; provides org.apache.lucene.index.SortFieldProvider with org.apache.lucene.search.SortField.Provider, org.apache.lucene.search.SortedNumericSortField.Provider, diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java index 0b654a134a6..e5a5dac8ff5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java @@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI { return LOADER; } - static Codec defaultCodec = LOADER.lookup("Lucene99"); + static Codec defaultCodec = LOADER.lookup("Lucene912"); } private final String name; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java b/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java index 37b9fc8bd20..77d38d290f8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java @@ -18,8 +18,6 @@ package org.apache.lucene.codecs; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -106,7 +104,7 @@ public final class CompetitiveImpactAccumulator { } /** Get the set of competitive freq and norm pairs, ordered by increasing freq and norm. */ - public Collection getCompetitiveFreqNormPairs() { + public List getCompetitiveFreqNormPairs() { List impacts = new ArrayList<>(); int maxFreqForLowerNorms = 0; for (int i = 0; i < maxFreqs.length; ++i) { @@ -126,7 +124,7 @@ public final class CompetitiveImpactAccumulator { for (Impact impact : impacts) { add(impact, freqNormPairs); } - return Collections.unmodifiableSet(freqNormPairs); + return List.copyOf(freqNormPairs); } private void add(Impact newEntry, TreeSet freqNormPairs) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java index 5646724546a..ce0310d6396 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java @@ -49,9 +49,9 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter; * *
      *   // the default: for high performance
    - *   indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_SPEED));
    + *   indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_SPEED));
      *   // instead for higher performance (but slower):
    - *   // indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_COMPRESSION));
    + *   // indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_COMPRESSION));
      * 
    * *

    File formats diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java new file mode 100644 index 00000000000..8b9aedcfb2b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java @@ -0,0 +1,83 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.packed.PackedInts; + +/** Utility class to encode/decode increasing sequences of 128 integers. */ +public class ForDeltaUtil { + + // IDENTITY_PLUS_ONE[i] == i+1 + private static final long[] IDENTITY_PLUS_ONE = new long[ForUtil.BLOCK_SIZE]; + + static { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + IDENTITY_PLUS_ONE[i] = i + 1; + } + } + + private static void prefixSumOfOnes(long[] arr, long base) { + System.arraycopy(IDENTITY_PLUS_ONE, 0, arr, 0, ForUtil.BLOCK_SIZE); + // This loop gets auto-vectorized + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + arr[i] += base; + } + } + + private final ForUtil forUtil; + + ForDeltaUtil(ForUtil forUtil) { + this.forUtil = forUtil; + } + + /** + * Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code + * longs} are expected to be deltas between consecutive values. + */ + void encodeDeltas(long[] longs, DataOutput out) throws IOException { + if (longs[0] == 1 && PForUtil.allEqual(longs)) { // happens with very dense postings + out.writeByte((byte) 0); + } else { + long or = 0; + for (long l : longs) { + or |= l; + } + assert or != 0; + final int bitsPerValue = PackedInts.bitsRequired(or); + out.writeByte((byte) bitsPerValue); + forUtil.encode(longs, bitsPerValue, out); + } + } + + /** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */ + void decodeAndPrefixSum(DataInput in, long base, long[] longs) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); + if (bitsPerValue == 0) { + prefixSumOfOnes(longs, base); + } else { + forUtil.decodeAndPrefixSum(bitsPerValue, in, base, longs); + } + } + + void skip(DataInput in) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); + in.skipBytes(forUtil.numBytes(bitsPerValue)); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java index 3126cdb02c0..63ee7baaf10 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java new file mode 100644 index 00000000000..cb4ef755a6b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java @@ -0,0 +1,217 @@ +/* + * 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.codecs.lucene912; + +import java.util.Objects; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PointsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; +import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; +import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; +import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; +import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; +import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; + +/** + * Implements the Lucene 9.12 index format + * + *

    If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. + * + * @see org.apache.lucene.codecs.lucene99 package documentation for file format details. + * @lucene.experimental + */ +public class Lucene912Codec extends Codec { + + /** Configuration option for the codec. */ + public enum Mode { + /** Trade compression ratio for retrieval speed. */ + BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED), + /** Trade retrieval speed for compression ratio. */ + BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION); + + private final Lucene90StoredFieldsFormat.Mode storedMode; + + private Mode(Lucene90StoredFieldsFormat.Mode storedMode) { + this.storedMode = Objects.requireNonNull(storedMode); + } + } + + private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat(); + private final FieldInfosFormat fieldInfosFormat = new Lucene94FieldInfosFormat(); + private final SegmentInfoFormat segmentInfosFormat = new Lucene99SegmentInfoFormat(); + private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat(); + private final CompoundFormat compoundFormat = new Lucene90CompoundFormat(); + private final NormsFormat normsFormat = new Lucene90NormsFormat(); + + private final PostingsFormat defaultPostingsFormat; + private final PostingsFormat postingsFormat = + new PerFieldPostingsFormat() { + @Override + public PostingsFormat getPostingsFormatForField(String field) { + return Lucene912Codec.this.getPostingsFormatForField(field); + } + }; + + private final DocValuesFormat defaultDVFormat; + private final DocValuesFormat docValuesFormat = + new PerFieldDocValuesFormat() { + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return Lucene912Codec.this.getDocValuesFormatForField(field); + } + }; + + private final KnnVectorsFormat defaultKnnVectorsFormat; + private final KnnVectorsFormat knnVectorsFormat = + new PerFieldKnnVectorsFormat() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return Lucene912Codec.this.getKnnVectorsFormatForField(field); + } + }; + + private final StoredFieldsFormat storedFieldsFormat; + + /** Instantiates a new codec. */ + public Lucene912Codec() { + this(Mode.BEST_SPEED); + } + + /** + * Instantiates a new codec, specifying the stored fields compression mode to use. + * + * @param mode stored fields compression mode to use for newly flushed/merged segments. + */ + public Lucene912Codec(Mode mode) { + super("Lucene912"); + this.storedFieldsFormat = + new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); + this.defaultPostingsFormat = new Lucene912PostingsFormat(); + this.defaultDVFormat = new Lucene90DocValuesFormat(); + this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat(); + } + + @Override + public final StoredFieldsFormat storedFieldsFormat() { + return storedFieldsFormat; + } + + @Override + public final TermVectorsFormat termVectorsFormat() { + return vectorsFormat; + } + + @Override + public final PostingsFormat postingsFormat() { + return postingsFormat; + } + + @Override + public final FieldInfosFormat fieldInfosFormat() { + return fieldInfosFormat; + } + + @Override + public final SegmentInfoFormat segmentInfoFormat() { + return segmentInfosFormat; + } + + @Override + public final LiveDocsFormat liveDocsFormat() { + return liveDocsFormat; + } + + @Override + public final CompoundFormat compoundFormat() { + return compoundFormat; + } + + @Override + public final PointsFormat pointsFormat() { + return new Lucene90PointsFormat(); + } + + @Override + public final KnnVectorsFormat knnVectorsFormat() { + return knnVectorsFormat; + } + + /** + * Returns the postings format that should be used for writing new segments of field. + * + *

    The default implementation always returns "Lucene912". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation, + */ + public PostingsFormat getPostingsFormatForField(String field) { + return defaultPostingsFormat; + } + + /** + * Returns the docvalues format that should be used for writing new segments of field + * . + * + *

    The default implementation always returns "Lucene90". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public DocValuesFormat getDocValuesFormatForField(String field) { + return defaultDVFormat; + } + + /** + * Returns the vectors format that should be used for writing new segments of field + * + *

    The default implementation always returns "Lucene99HnswVectorsFormat". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return defaultKnnVectorsFormat; + } + + @Override + public final DocValuesFormat docValuesFormat() { + return docValuesFormat; + } + + @Override + public final NormsFormat normsFormat() { + return normsFormat; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java new file mode 100644 index 00000000000..1c452175b07 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java @@ -0,0 +1,492 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.TermState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.packed.PackedInts; + +/** + * Lucene 9.12 postings format, which encodes postings in packed integer blocks for fast decode. + * + *

    Basic idea: + * + *

      + *
    • Packed Blocks and VInt Blocks: + *

      In packed blocks, integers are encoded with the same bit width ({@link PackedInts packed + * format}): the block size (i.e. number of integers inside block) is fixed (currently 128). + * Additionally blocks that are all the same value are encoded in an optimized way. + *

      In VInt blocks, integers are encoded as {@link DataOutput#writeVInt VInt}: the block + * size is variable. + *

    • Block structure: + *

      When the postings are long enough, Lucene912PostingsFormat will try to encode most + * integer data as a packed block. + *

      Take a term with 259 documents as an example, the first 256 document ids are encoded as + * two packed blocks, while the remaining 3 are encoded as one VInt block. + *

      Different kinds of data are always encoded separately into different packed blocks, but + * may possibly be interleaved into the same VInt block. + *

      This strategy is applied to pairs: <document number, frequency>, <position, + * payload length>, <position, offset start, offset length>, and <position, + * payload length, offsetstart, offset length>. + *

    • Skipdata: + *

      Skipdata is interleaved with blocks on 2 levels. Level 0 skip data is interleaved + * between every packed block. Level 1 skip data is interleaved between every 32 packed + * blocks. + *

    • Positions, Payloads, and Offsets: + *

      A position is an integer indicating where the term occurs within one document. A payload + * is a blob of metadata associated with current position. An offset is a pair of integers + * indicating the tokenized start/end offsets for given term in current position: it is + * essentially a specialized payload. + *

      When payloads and offsets are not omitted, numPositions==numPayloads==numOffsets + * (assuming a null payload contributes one count). As mentioned in block structure, it is + * possible to encode these three either combined or separately. + *

      In all cases, payloads and offsets are stored together. When encoded as a packed block, + * position data is separated out as .pos, while payloads and offsets are encoded in .pay + * (payload metadata will also be stored directly in .pay). When encoded as VInt blocks, all + * these three are stored interleaved into the .pos (so is payload metadata). + *

      With this strategy, the majority of payload and offset data will be outside .pos file. + * So for queries that require only position data, running on a full index with payloads and + * offsets, this reduces disk pre-fetches. + *

    + * + *

    Files and detailed format: + * + *

    + * + * + * + *
    + *
    Term Dictionary + *

    The .tim file contains the list of terms in each field along with per-term statistics + * (such as docfreq) and pointers to the frequencies, positions, payload and skip data in the + * .doc, .pos, and .pay files. See {@link Lucene90BlockTreeTermsWriter} for more details on + * the format. + *

    NOTE: The term dictionary can plug into different postings implementations: the postings + * writer/reader are actually responsible for encoding and decoding the PostingsHeader and + * TermMetadata sections described here: + *

      + *
    • PostingsHeader --> Header, PackedBlockSize + *
    • TermMetadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, + * PayFPDelta? + *
    • Header, --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • PackedBlockSize, SingletonDocID --> {@link DataOutput#writeVInt VInt} + *
    • DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta --> {@link + * DataOutput#writeVLong VLong} + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • Header is a {@link CodecUtil#writeIndexHeader IndexHeader} storing the version + * information for the postings. + *
    • PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width + * is determined by the largest integer. Smaller block size result in smaller variance + * among width of integers hence smaller indexes. Larger block size result in more + * efficient bulk i/o hence better acceleration. This value should always be a multiple + * of 64, currently fixed as 128 as a tradeoff. It is also the skip interval used to + * accelerate {@link org.apache.lucene.index.PostingsEnum#advance(int)}. + *
    • DocFPDelta determines the position of this term's TermFreqs within the .doc file. In + * particular, it is the difference of file offset between this term's data and previous + * term's data (or zero, for the first term in the block).On disk it is stored as the + * difference from previous value in sequence. + *
    • PosFPDelta determines the position of this term's TermPositions within the .pos file. + * While PayFPDelta determines the position of this term's <TermPayloads, + * TermOffsets?> within the .pay file. Similar to DocFPDelta, it is the difference + * between two file positions (or neglected, for fields that omit payloads and offsets). + *
    • PosVIntBlockFPDelta determines the position of this term's last TermPosition in last + * pos packed block within the .pos file. It is synonym for PayVIntBlockFPDelta or + * OffsetVIntBlockFPDelta. This is actually used to indicate whether it is necessary to + * load following payloads and offsets from .pos instead of .pay. Every time a new block + * of positions are to be loaded, the PostingsReader will use this value to check + * whether current block is packed format or VInt. When packed format, payloads and + * offsets are fetched from .pay, otherwise from .pos. (this value is neglected when + * total number of positions i.e. totalTermFreq is less or equal to PackedBlockSize). + *
    • SingletonDocID is an optimization when a term only appears in one document. In this + * case, instead of writing a file pointer to the .doc file (DocFPDelta), and then a + * VIntBlock at that location, the single document ID is written to the term dictionary. + *
    + *
    + * + * + * + *
    + *
    Term Index + *

    The .tip file contains an index into the term dictionary, so that it can be accessed + * randomly. See {@link Lucene90BlockTreeTermsWriter} for more details on the format. + *

    + * + * + * + *
    + *
    Frequencies and Skip Data + *

    The .doc file contains the lists of documents which contain each term, along with the + * frequency of the term in that document (except when frequencies are omitted: {@link + * IndexOptions#DOCS}). Skip data is saved at the end of each term's postings. The skip data + * is saved once for the entire postings list. + *

      + *
    • docFile(.doc) --> Header, <TermFreqs>TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermFreqs --> <PackedBlock32> PackedDocBlockNum/32, VIntBlock? + *
    • PackedBlock32 --> Level1SkipData, <PackedBlock> 32 + *
    • PackedBlock --> Level0SkipData, PackedDocDeltaBlock, PackedFreqBlock? + *
    • VIntBlock --> + * <DocDelta[,Freq?]>DocFreq-PackedBlockSize*PackedDocBlockNum + *
    • Level1SkipData --> DocDelta, DocFPDelta, Skip1NumBytes?, ImpactLength?, Impacts?, + * PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto? + *
    • Level0SkipData --> Skip0NumBytes, DocDelta, DocFPDelta, PackedBlockLength, + * ImpactLength?, Impacts?, PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto? + *
    • PackedFreqBlock --> {@link PackedInts PackedInts}, uses patching + *
    • PackedDocDeltaBlock --> {@link PackedInts PackedInts}, does not use patching + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • PackedDocDeltaBlock is theoretically generated from two steps: + *
        + *
      1. Calculate the difference between each document number and previous one, and get + * a d-gaps list (for the first document, use absolute value); + *
      2. For those d-gaps from first one to + * PackedDocBlockNum*PackedBlockSizeth, separately encode as packed + * blocks. + *
      + * If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step. + *
    • VIntBlock stores remaining d-gaps (along with frequencies when possible) with a + * format that encodes DocDelta and Freq: + *

      DocDelta: if frequencies are indexed, this determines both the document number and + * the frequency. In particular, DocDelta/2 is the difference between this document + * number and the previous document number (or zero when this is the first document in a + * TermFreqs). When DocDelta is odd, the frequency is one. When DocDelta is even, the + * frequency is read as another VInt. If frequencies are omitted, DocDelta contains the + * gap (not multiplied by 2) between document numbers and no frequency information is + * stored. + *

      For example, the TermFreqs for a term which occurs once in document seven and + * three times in document eleven, with frequencies indexed, would be the following + * sequence of VInts: + *

      15, 8, 3 + *

      If frequencies were omitted ({@link IndexOptions#DOCS}) it would be this sequence + * of VInts instead: + *

      7,4 + *

    • PackedDocBlockNum is the number of packed blocks for current term's docids or + * frequencies. In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize) + *
    • On skip data, DocDelta is the delta between the last doc of the previous block - or + * -1 if there is no previous block - and the last doc of this block. This helps know by + * how much the doc ID should be incremented in case the block gets skipped. + *
    • Skip0Length is the length of skip data at level 0. Encoding it is useful when skip + * data is never needed to quickly skip over skip data, e.g. if only using nextDoc(). It + * is also used when only the first fields of skip data are needed, in order to skip + * over remaining fields without reading them. + *
    • ImpactLength and Impacts are only stored if frequencies are indexed. + *
    • Since positions and payloads are also block encoded, the skip should skip to related + * block first, then fetch the values according to in-block offset. PosFPSkip and + * PayFPSkip record the file offsets of related block in .pos and .pay, respectively. + * While PosBlockOffset indicates which value to fetch inside the related block + * (PayBlockOffset is unnecessary since it is always equal to PosBlockOffset). Same as + * DocFPSkip, the file offsets are relative to the start of current term's TermFreqs, + * and stored as a difference sequence. + *
    • PayByteUpto indicates the start offset of the current payload. It is equivalent to + * the sum of the payload lengths in the current block up to PosBlockOffset + *
    • ImpactLength is the total length of CompetitiveFreqDelta and CompetitiveNormDelta + * pairs. CompetitiveFreqDelta and CompetitiveNormDelta are used to safely skip score + * calculation for uncompetitive documents; See {@link + * org.apache.lucene.codecs.CompetitiveImpactAccumulator} for more details. + *
    + *
    + * + * + * + *
    + *
    Positions + *

    The .pos file contains the lists of positions that each term occurs at within documents. + * It also sometimes stores part of payloads and offsets for speedup. + *

      + *
    • PosFile(.pos) --> Header, <TermPositions> TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPositions --> <PackedPosDeltaBlock> PackedPosBlockNum, + * VIntBlock? + *
    • VIntBlock --> <PositionDelta[, PayloadLength?], PayloadData?, OffsetDelta?, + * OffsetLength?>PosVIntCount + *
    • PackedPosDeltaBlock --> {@link PackedInts PackedInts} + *
    • PositionDelta, OffsetDelta, OffsetLength --> {@link DataOutput#writeVInt VInt} + *
    • PayloadData --> {@link DataOutput#writeByte byte}PayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • TermPositions are order by term (terms are implicit, from the term dictionary), and + * position values for each term document pair are incremental, and ordered by document + * number. + *
    • PackedPosBlockNum is the number of packed blocks for current term's positions, + * payloads or offsets. In particular, PackedPosBlockNum = + * floor(totalTermFreq/PackedBlockSize) + *
    • PosVIntCount is the number of positions encoded as VInt format. In particular, + * PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize + *
    • The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock + * in chapter Frequencies and Skip Data. + *
    • PositionDelta is, if payloads are disabled for the term's field, the difference + * between the position of the current occurrence in the document and the previous + * occurrence (or zero, if this is the first occurrence in this document). If payloads + * are enabled for the term's field, then PositionDelta/2 is the difference between the + * current and the previous position. If payloads are enabled and PositionDelta is odd, + * then PayloadLength is stored, indicating the length of the payload at the current + * term position. + *
    • For example, the TermPositions for a term which occurs as the fourth term in one + * document, and as the fifth and ninth term in a subsequent document, would be the + * following sequence of VInts (payloads disabled): + *

      4, 5, 4 + *

    • PayloadData is metadata associated with the current term position. If PayloadLength + * is stored at the current position, then it indicates the length of this payload. If + * PayloadLength is not stored, then this payload has the same length as the payload at + * the previous position. + *
    • OffsetDelta/2 is the difference between this position's startOffset from the previous + * occurrence (or zero, if this is the first occurrence in this document). If + * OffsetDelta is odd, then the length (endOffset-startOffset) differs from the previous + * occurrence and an OffsetLength follows. Offset data is only written for {@link + * IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}. + *
    + *
    + * + * + * + *
    + *
    Payloads and Offsets + *

    The .pay file will store payloads and offsets associated with certain term-document + * positions. Some payloads and offsets will be separated out into .pos file, for performance + * reasons. + *

      + *
    • PayFile(.pay): --> Header, <TermPayloads?, TermOffsets?> + * TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPayloads --> <PackedPayLengthBlock, SumPayLength, PayData> + * PackedPayBlockNum + *
    • TermOffsets --> <PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock> + * PackedPayBlockNum + *
    • PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --> + * {@link PackedInts PackedInts} + *
    • SumPayLength --> {@link DataOutput#writeVInt VInt} + *
    • PayData --> {@link DataOutput#writeByte byte}SumPayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • The order of TermPayloads/TermOffsets will be the same as TermPositions, note that + * part of payload/offsets are stored in .pos. + *
    • The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is + * the same as PackedFreqBlock in chapter Frequencies and Skip + * Data. While PackedStartDeltaBlock follows a same procedure as + * PackedDocDeltaBlock. + *
    • PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also + * synonym for PackedOffsetBlockNum. + *
    • SumPayLength is the total length of payloads written within one block, should be the + * sum of PayLengths in one packed block. + *
    • PayLength in PackedPayLengthBlock is the length of each payload associated with the + * current position. + *
    + *
    + * + * @lucene.experimental + */ +public final class Lucene912PostingsFormat extends PostingsFormat { + + /** Filename extension for some small metadata about how postings are encoded. */ + public static final String META_EXTENSION = "psm"; + + /** + * Filename extension for document number, frequencies, and skip data. See chapter: Frequencies and Skip Data + */ + public static final String DOC_EXTENSION = "doc"; + + /** Filename extension for positions. See chapter: Positions */ + public static final String POS_EXTENSION = "pos"; + + /** + * Filename extension for payloads and offsets. See chapter: Payloads and + * Offsets + */ + public static final String PAY_EXTENSION = "pay"; + + /** Size of blocks. */ + public static final int BLOCK_SIZE = ForUtil.BLOCK_SIZE; + + public static final int BLOCK_MASK = BLOCK_SIZE - 1; + + /** We insert skip data on every block and every SKIP_FACTOR=32 blocks. */ + public static final int LEVEL1_FACTOR = 32; + + /** Total number of docs covered by level 1 skip data: 32 * 128 = 4,096 */ + public static final int LEVEL1_NUM_DOCS = LEVEL1_FACTOR * BLOCK_SIZE; + + public static final int LEVEL1_MASK = LEVEL1_NUM_DOCS - 1; + + static final String TERMS_CODEC = "Lucene90PostingsWriterTerms"; + static final String META_CODEC = "Lucene912PostingsWriterMeta"; + static final String DOC_CODEC = "Lucene912PostingsWriterDoc"; + static final String POS_CODEC = "Lucene912PostingsWriterPos"; + static final String PAY_CODEC = "Lucene912PostingsWriterPay"; + + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + private final int minTermBlockSize; + private final int maxTermBlockSize; + + /** Creates {@code Lucene912PostingsFormat} with default settings. */ + public Lucene912PostingsFormat() { + this( + Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + /** + * Creates {@code Lucene912PostingsFormat} with custom values for {@code minBlockSize} and {@code + * maxBlockSize} passed to block terms dictionary. + * + * @see + * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) + */ + public Lucene912PostingsFormat(int minTermBlockSize, int maxTermBlockSize) { + super("Lucene912"); + Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); + this.minTermBlockSize = minTermBlockSize; + this.maxTermBlockSize = maxTermBlockSize; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + boolean success = false; + try { + FieldsConsumer ret = + new Lucene90BlockTreeTermsWriter( + state, postingsWriter, minTermBlockSize, maxTermBlockSize); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsWriter); + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); + boolean success = false; + try { + FieldsProducer ret = new Lucene90BlockTreeTermsReader(postingsReader, state); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsReader); + } + } + } + + /** + * Holds all state required for {@link Lucene912PostingsReader} to produce a {@link + * org.apache.lucene.index.PostingsEnum} without re-seeking the terms dict. + * + * @lucene.internal + */ + public static final class IntBlockTermState extends BlockTermState { + /** file pointer to the start of the doc ids enumeration, in {@link #DOC_EXTENSION} file */ + public long docStartFP; + + /** file pointer to the start of the positions enumeration, in {@link #POS_EXTENSION} file */ + public long posStartFP; + + /** file pointer to the start of the payloads enumeration, in {@link #PAY_EXTENSION} file */ + public long payStartFP; + + /** + * file offset for the last position in the last block, if there are more than {@link + * ForUtil#BLOCK_SIZE} positions; otherwise -1 + * + *

    One might think to use total term frequency to track how many positions are left to read + * as we decode the blocks, and decode the last block differently when num_left_positions < + * BLOCK_SIZE. Unfortunately this won't work since the tracking will be messed up when we skip + * blocks as the skipper will only tell us new position offset (start of block) and number of + * positions to skip for that block, without telling us how many positions it has skipped. + */ + public long lastPosBlockOffset; + + /** + * docid when there is a single pulsed posting, otherwise -1. freq is always implicitly + * totalTermFreq in this case. + */ + public int singletonDocID; + + /** Sole constructor. */ + public IntBlockTermState() { + lastPosBlockOffset = -1; + singletonDocID = -1; + } + + @Override + public IntBlockTermState clone() { + IntBlockTermState other = new IntBlockTermState(); + other.copyFrom(this); + return other; + } + + @Override + public void copyFrom(TermState _other) { + super.copyFrom(_other); + IntBlockTermState other = (IntBlockTermState) _other; + docStartFP = other.docStartFP; + posStartFP = other.posStartFP; + payStartFP = other.payStartFP; + lastPosBlockOffset = other.lastPosBlockOffset; + singletonDocID = other.singletonDocID; + } + + @Override + public String toString() { + return super.toString() + + " docStartFP=" + + docStartFP + + " posStartFP=" + + posStartFP + + " payStartFP=" + + payStartFP + + " lastPosBlockOffset=" + + lastPosBlockOffset + + " singletonDocID=" + + singletonDocID; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java new file mode 100644 index 00000000000..f0f22f42663 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -0,0 +1,2104 @@ +/* + * 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.codecs.lucene912; + +import static org.apache.lucene.codecs.lucene912.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_NUM_DOCS; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_START; + +import java.io.IOException; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.RandomAccess; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.Impacts; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SlowImpactsEnum; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.ReadAdvice; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.IOUtils; + +/** + * Concrete class that reads docId(maybe frq,pos,offset,payloads) list with postings format. + * + * @lucene.experimental + */ +public final class Lucene912PostingsReader extends PostingsReaderBase { + + private final IndexInput docIn; + private final IndexInput posIn; + private final IndexInput payIn; + + private final int maxNumImpactsAtLevel0; + private final int maxImpactNumBytesAtLevel0; + private final int maxNumImpactsAtLevel1; + private final int maxImpactNumBytesAtLevel1; + + private final int version; + + /** Sole constructor. */ + public Lucene912PostingsReader(SegmentReadState state) throws IOException { + String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.META_EXTENSION); + final long expectedDocFileLength, expectedPosFileLength, expectedPayFileLength; + ChecksumIndexInput metaIn = null; + boolean success = false; + try { + metaIn = state.directory.openChecksumInput(metaName); + version = + CodecUtil.checkIndexHeader( + metaIn, + META_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + maxNumImpactsAtLevel0 = metaIn.readInt(); + maxImpactNumBytesAtLevel0 = metaIn.readInt(); + maxNumImpactsAtLevel1 = metaIn.readInt(); + maxImpactNumBytesAtLevel1 = metaIn.readInt(); + expectedDocFileLength = metaIn.readLong(); + if (state.fieldInfos.hasProx()) { + expectedPosFileLength = metaIn.readLong(); + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + expectedPayFileLength = metaIn.readLong(); + } else { + expectedPayFileLength = -1; + } + } else { + expectedPosFileLength = -1; + expectedPayFileLength = -1; + } + CodecUtil.checkFooter(metaIn, null); + success = true; + } catch (Throwable t) { + if (metaIn != null) { + CodecUtil.checkFooter(metaIn, t); + throw new AssertionError("unreachable"); + } else { + throw t; + } + } finally { + if (success) { + metaIn.close(); + } else { + IOUtils.closeWhileHandlingException(metaIn); + } + } + + success = false; + IndexInput docIn = null; + IndexInput posIn = null; + IndexInput payIn = null; + + // NOTE: these data files are too costly to verify checksum against all the bytes on open, + // but for now we at least verify proper structure of the checksum footer: which looks + // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption + // such as file truncation. + + String docName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.DOC_EXTENSION); + try { + // Postings have a forward-only access pattern, so pass ReadAdvice.NORMAL to perform + // readahead. + docIn = state.directory.openInput(docName, state.context.withReadAdvice(ReadAdvice.NORMAL)); + CodecUtil.checkIndexHeader( + docIn, DOC_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(docIn, expectedDocFileLength); + + if (state.fieldInfos.hasProx()) { + String proxName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.POS_EXTENSION); + posIn = state.directory.openInput(proxName, state.context); + CodecUtil.checkIndexHeader( + posIn, POS_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(posIn, expectedPosFileLength); + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene912PostingsFormat.PAY_EXTENSION); + payIn = state.directory.openInput(payName, state.context); + CodecUtil.checkIndexHeader( + payIn, PAY_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(payIn, expectedPayFileLength); + } + } + + this.docIn = docIn; + this.posIn = posIn; + this.payIn = payIn; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(docIn, posIn, payIn); + } + } + } + + @Override + public void init(IndexInput termsIn, SegmentReadState state) throws IOException { + // Make sure we are talking to the matching postings writer + CodecUtil.checkIndexHeader( + termsIn, + TERMS_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + final int indexBlockSize = termsIn.readVInt(); + if (indexBlockSize != BLOCK_SIZE) { + throw new IllegalStateException( + "index-time BLOCK_SIZE (" + + indexBlockSize + + ") != read-time BLOCK_SIZE (" + + BLOCK_SIZE + + ")"); + } + } + + static void prefixSum(long[] buffer, int count, long base) { + buffer[0] += base; + for (int i = 1; i < count; ++i) { + buffer[i] += buffer[i - 1]; + } + } + + static int findFirstGreater(long[] buffer, int target, int from) { + for (int i = from; i < BLOCK_SIZE; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return BLOCK_SIZE; + } + + @Override + public BlockTermState newTermState() { + return new IntBlockTermState(); + } + + @Override + public void close() throws IOException { + IOUtils.close(docIn, posIn, payIn); + } + + @Override + public void decodeTerm( + DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute) + throws IOException { + final IntBlockTermState termState = (IntBlockTermState) _termState; + final boolean fieldHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean fieldHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean fieldHasPayloads = fieldInfo.hasPayloads(); + + if (absolute) { + termState.docStartFP = 0; + termState.posStartFP = 0; + termState.payStartFP = 0; + } + + final long l = in.readVLong(); + if ((l & 0x01) == 0) { + termState.docStartFP += l >>> 1; + if (termState.docFreq == 1) { + termState.singletonDocID = in.readVInt(); + } else { + termState.singletonDocID = -1; + } + } else { + assert absolute == false; + assert termState.singletonDocID != -1; + termState.singletonDocID += BitUtil.zigZagDecode(l >>> 1); + } + + if (fieldHasPositions) { + termState.posStartFP += in.readVLong(); + if (fieldHasOffsets || fieldHasPayloads) { + termState.payStartFP += in.readVLong(); + } + if (termState.totalTermFreq > BLOCK_SIZE) { + termState.lastPosBlockOffset = in.readVLong(); + } else { + termState.lastPosBlockOffset = -1; + } + } + } + + @Override + public PostingsEnum postings( + FieldInfo fieldInfo, BlockTermState termState, PostingsEnum reuse, int flags) + throws IOException { + + boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false) { + BlockDocsEnum docsEnum; + if (reuse instanceof BlockDocsEnum) { + docsEnum = (BlockDocsEnum) reuse; + if (!docsEnum.canReuse(docIn, fieldInfo)) { + docsEnum = new BlockDocsEnum(fieldInfo); + } + } else { + docsEnum = new BlockDocsEnum(fieldInfo); + } + return docsEnum.reset((IntBlockTermState) termState, flags); + } else { + EverythingEnum everythingEnum; + if (reuse instanceof EverythingEnum) { + everythingEnum = (EverythingEnum) reuse; + if (!everythingEnum.canReuse(docIn, fieldInfo)) { + everythingEnum = new EverythingEnum(fieldInfo); + } + } else { + everythingEnum = new EverythingEnum(fieldInfo); + } + return everythingEnum.reset((IntBlockTermState) termState, flags); + } + } + + @Override + public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) + throws IOException { + final boolean indexHasFreqs = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + final boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (state.docFreq >= BLOCK_SIZE + && indexHasFreqs + && (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false)) { + return new BlockImpactsDocsEnum(fieldInfo, (IntBlockTermState) state); + } + + final boolean indexHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean indexHasPayloads = fieldInfo.hasPayloads(); + + if (state.docFreq >= BLOCK_SIZE + && indexHasPositions + && (indexHasOffsets == false + || PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) == false) + && (indexHasPayloads == false + || PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) == false)) { + return new BlockImpactsPostingsEnum(fieldInfo, (IntBlockTermState) state); + } + + return new SlowImpactsEnum(postings(fieldInfo, state, null, flags)); + } + + final class BlockDocsEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + + // level 0 skip data + private int level0LastDocID; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndOffset; + private int level1DocCountUpto; + + private boolean needsFreq; // true if the caller actually needs frequencies + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + private long freqFP; + + public BlockDocsEnum(FieldInfo fieldInfo) throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsetsOrPayloads = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads(); + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && indexHasFreq + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) + && indexHasPos + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) + >= 0) + && indexHasOffsetsOrPayloads + == (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads()); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + docFreq = termState.docFreq; + totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; + singletonDocID = termState.singletonDocID; + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + + doc = -1; + this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS); + if (indexHasFreq == false || needsFreq == false) { + // Filling this buffer may not be cheap when doing primary key lookups, so we make sure to + // not fill more than `docFreq` entries. + Arrays.fill(freqBuffer, 0, Math.min(ForUtil.BLOCK_SIZE, docFreq), 1); + } + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndOffset = termState.docStartFP; + level1DocCountUpto = 0; + docBufferUpto = BLOCK_SIZE; + freqFP = -1; + return this; + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docIn, freqBuffer); + } + + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() throws IOException { + return -1; + } + + @Override + public int endOffset() throws IOException { + return -1; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + @Override + public int docID() { + return doc; + } + + private void refillFullBlock() throws IOException { + assert docFreq - docCountUpto >= BLOCK_SIZE; + + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + + if (indexHasFreq) { + if (needsFreq) { + freqFP = docIn.getFilePointer(); + } + pforUtil.skip(docIn); + } + docCountUpto += BLOCK_SIZE; + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + private void refillRemainder() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + assert left < BLOCK_SIZE; + + if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, needsFreq); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + } + docBufferUpto = 0; + freqFP = -1; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndOffset); + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndOffset = docIn.readVLong() + docIn.getFilePointer(); + + if (level1LastDocID >= target) { + if (indexHasFreq) { + // skip impacts and pos skip data + docIn.skipBytes(docIn.readShort()); + } + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + long skip0NumBytes = docIn.readVLong(); + // end offset of skip data (before the actual data starts) + long skip0EndFP = docIn.getFilePointer() + skip0NumBytes; + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + docIn.seek(skip0EndFP); + break; + } + + // skip block + docIn.skipBytes(readVLong15(docIn)); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { // advance skip data on level 1 + skipLevel1To(doc + 1); + } + + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.skipBytes(docIn.readVLong()); + refillFullBlock(); + level0LastDocID = (int) docBuffer[BLOCK_SIZE - 1]; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + refillRemainder(); + } + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { // advance skip data on level 0 + moveToNextLevel0Block(); + } + + return this.doc = (int) docBuffer[docBufferUpto++]; + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID) { // advance skip data on level 0 + + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } + + skipLevel0To(target); + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + refillFullBlock(); + } else { + refillRemainder(); + } + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + this.doc = (int) docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class EverythingEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE + 1]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private final long[] payloadLengthBuffer; + private final long[] offsetStartDeltaBuffer; + private final long[] offsetLengthBuffer; + + private byte[] payloadBytes; + private int payloadByteUpto; + private int payloadLength; + + private int lastStartOffset; + private int startOffset; + private int endOffset; + + private int docBufferUpto; + private int posBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final IndexInput posIn; + final IndexInput payIn; + final BytesRef payload; + + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private long posPendingCount; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // Where this term's payloads/offsets start in the .pay + // file: + private long payTermStartFP; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + // level 0 skip data + private int level0LastDocID; + private long level0PosEndFP; + private int level0BlockPosUpto; + private long levelPayEndFP; + private int level0BlockPayUpto; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + private long level1PosEndFP; + private int level1BlockPosUpto; + private long level1PayEndFP; + private int level1BlockPayUpto; + + private boolean needsOffsets; // true if we actually need offsets + private boolean needsPayloads; // true if we actually need payloads + + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public EverythingEnum(FieldInfo fieldInfo) throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene912PostingsReader.this.posIn.clone(); + if (indexHasOffsetsOrPayloads) { + this.payIn = Lucene912PostingsReader.this.payIn.clone(); + } else { + this.payIn = null; + } + if (indexHasOffsets) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + startOffset = -1; + endOffset = -1; + } + + if (indexHasPayloads) { + payloadLengthBuffer = new long[BLOCK_SIZE]; + payloadBytes = new byte[128]; + payload = new BytesRef(); + } else { + payloadLengthBuffer = null; + payloadBytes = null; + payload = null; + } + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && indexHasOffsets + == (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0) + && indexHasPayloads == fieldInfo.hasPayloads(); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + docFreq = termState.docFreq; + posTermStartFP = termState.posStartFP; + payTermStartFP = termState.payStartFP; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + posIn.seek(posTermStartFP); + if (indexHasOffsetsOrPayloads) { + payIn.seek(payTermStartFP); + } + level1PosEndFP = posTermStartFP; + level1PayEndFP = payTermStartFP; + level0PosEndFP = posTermStartFP; + levelPayEndFP = payTermStartFP; + posPendingCount = 0; + payloadByteUpto = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + this.needsOffsets = PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS); + this.needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS); + + doc = -1; + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + level1BlockPosUpto = 0; + level1BlockPayUpto = 0; + level0BlockPosUpto = 0; + level0BlockPayUpto = 0; + docBufferUpto = BLOCK_SIZE; + posBufferUpto = BLOCK_SIZE; + return this; + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + level0PosEndFP = level1PosEndFP; + level0BlockPosUpto = level1BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + levelPayEndFP = level1PayEndFP; + level0BlockPayUpto = level1BlockPayUpto; + } + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + long delta = docIn.readVLong(); + level1DocEndFP = delta + docIn.getFilePointer(); + + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + docIn.skipBytes(docIn.readShort()); // impacts + level1PosEndFP += docIn.readVLong(); + level1BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + level1PayEndFP += docIn.readVLong(); + level1BlockPayUpto = docIn.readVInt(); + } + assert docIn.getFilePointer() == skip1EndFP; + + if (level1LastDocID >= target) { + break; + } + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { // advance level 1 skip data + skipLevel1To(doc + 1); + } + + // Now advance level 0 skip data + prevDocID = level0LastDocID; + + assert docBufferUpto == BLOCK_SIZE; + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + assert levelPayEndFP >= payIn.getFilePointer(); + payIn.seek(levelPayEndFP); + payloadByteUpto = level0BlockPayUpto; + } + posBufferUpto = BLOCK_SIZE; + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + readVLong15(docIn); // block length + docIn.skipBytes(docIn.readVLong()); // impacts + + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + levelPayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + } + + refillDocs(); + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { // advance level 0 skip data + moveToNextLevel0Block(); + } + + this.doc = (int) docBuffer[docBufferUpto]; + this.freq = (int) freqBuffer[docBufferUpto]; + docBufferUpto++; + posPendingCount += freq; + position = 0; + lastStartOffset = 0; + return doc; + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + + // If nextBlockPosFP is less than the current FP, it means that the block of positions for + // the first docs of the next block are already decoded. In this case we just accumulate + // frequencies into posPendingCount instead of seeking backwards and decoding the same pos + // block again. + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + assert levelPayEndFP >= payIn.getFilePointer(); + payIn.seek(levelPayEndFP); + payloadByteUpto = level0BlockPayUpto; + } + posBufferUpto = BLOCK_SIZE; + } else { + for (int i = docBufferUpto; i < BLOCK_SIZE; ++i) { + posPendingCount += freqBuffer[i]; + } + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + + long blockLength = readVLong15(docIn); + long blockEndFP = docIn.getFilePointer() + blockLength; + docIn.skipBytes(docIn.readVLong()); // impacts + + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + levelPayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + + if (target <= level0LastDocID) { + break; + } + + docIn.seek(blockEndFP); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + } + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID) { // advance level 0 skip data + + if (target > level1LastDocID) { // advance level 1 skip data + skipLevel1To(target); + } + + skipLevel0To(target); + + refillDocs(); + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + for (int i = docBufferUpto; i <= next; ++i) { + posPendingCount += freqBuffer[i]; + } + this.freq = (int) freqBuffer[next]; + this.docBufferUpto = next + 1; + position = 0; + lastStartOffset = 0; + + return this.doc = (int) docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + long toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + int end = (int) (posBufferUpto + toSkip); + if (indexHasPayloads) { + for (int i = posBufferUpto; i < end; ++i) { + payloadByteUpto += payloadLengthBuffer[i]; + } + } + posBufferUpto = end; + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + pforUtil.skip(posIn); + + if (indexHasPayloads) { + // Skip payloadLength block: + pforUtil.skip(payIn); + + // Skip payloadBytes block: + int numBytes = payIn.readVInt(); + payIn.seek(payIn.getFilePointer() + numBytes); + } + + if (indexHasOffsets) { + pforUtil.skip(payIn); + pforUtil.skip(payIn); + } + toSkip -= BLOCK_SIZE; + } + refillPositions(); + payloadByteUpto = 0; + posBufferUpto = 0; + final int toSkipInt = (int) toSkip; + if (indexHasPayloads) { + for (int i = 0; i < toSkipInt; ++i) { + payloadByteUpto += payloadLengthBuffer[i]; + } + } + posBufferUpto = toSkipInt; + } + + position = 0; + lastStartOffset = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + int offsetLength = 0; + payloadByteUpto = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + payloadLengthBuffer[i] = payloadLength; + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + if (payloadByteUpto + payloadLength > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength); + } + posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength); + payloadByteUpto += payloadLength; + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + offsetLength = posIn.readVInt(); + } + offsetStartDeltaBuffer[i] = deltaCode >>> 1; + offsetLengthBuffer[i] = offsetLength; + } + } + payloadByteUpto = 0; + } else { + pforUtil.decode(posIn, posDeltaBuffer); + + if (indexHasPayloads) { + if (needsPayloads) { + pforUtil.decode(payIn, payloadLengthBuffer); + int numBytes = payIn.readVInt(); + + if (numBytes > payloadBytes.length) { + payloadBytes = ArrayUtil.growNoCopy(payloadBytes, numBytes); + } + payIn.readBytes(payloadBytes, 0, numBytes); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over lengths + int numBytes = payIn.readVInt(); // read length of payloadBytes + payIn.seek(payIn.getFilePointer() + numBytes); // skip over payloadBytes + } + payloadByteUpto = 0; + } + + if (indexHasOffsets) { + if (needsOffsets) { + pforUtil.decode(payIn, offsetStartDeltaBuffer); + pforUtil.decode(payIn, offsetLengthBuffer); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over starts + pforUtil.skip(payIn); // skip over lengths + } + } + } + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + if (indexHasPayloads) { + payloadLength = (int) payloadLengthBuffer[posBufferUpto]; + payload.bytes = payloadBytes; + payload.offset = payloadByteUpto; + payload.length = payloadLength; + payloadByteUpto += payloadLength; + } + + if (indexHasOffsets) { + startOffset = lastStartOffset + (int) offsetStartDeltaBuffer[posBufferUpto]; + endOffset = startOffset + (int) offsetLengthBuffer[posBufferUpto]; + lastStartOffset = startOffset; + } + + posBufferUpto++; + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return startOffset; + } + + @Override + public int endOffset() { + return endOffset; + } + + @Override + public BytesRef getPayload() { + if (payloadLength == 0) { + return null; + } else { + return payload; + } + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsDocsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + private long freqFP; + + // true if we shallow-advanced to a new block that we have not decoded yet + private boolean needsRefilling; + + // level 0 skip data + private int level0LastDocID; + private long level0DocEndFP; + private final BytesRef level0SerializedImpacts; + private final ByteArrayDataInput level0SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level0Impacts; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + private final BytesRef level1SerializedImpacts; + private final ByteArrayDataInput level1SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level1Impacts; + + public BlockImpactsDocsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsetsOrPayloads = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads(); + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + + docFreq = termState.docFreq; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + + doc = -1; + if (indexHasFreq == false) { + // Filling this buffer may not be cheap when doing primary key lookups, so we make sure to + // not fill more than `docFreq` entries. + Arrays.fill(freqBuffer, 0, Math.min(ForUtil.BLOCK_SIZE, docFreq), 1); + } + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + docBufferUpto = BLOCK_SIZE; + freqFP = -1; + level0SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel0); + level1SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel1); + level0Impacts = new MutableImpactList(maxNumImpactsAtLevel0); + level1Impacts = new MutableImpactList(maxNumImpactsAtLevel1); + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docIn, freqBuffer); + freqFP = -1; + } + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() throws IOException { + return -1; + } + + @Override + public int endOffset() throws IOException { + return -1; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + + if (indexHasFreq) { + freqFP = docIn.getFilePointer(); + pforUtil.skip(docIn); + } + docCountUpto += BLOCK_SIZE; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + freqFP = -1; + docCountUpto += left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); + + if (level1LastDocID >= target) { + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + int numImpactBytes = docIn.readShort(); + docIn.readBytes(level1SerializedImpacts.bytes, 0, numImpactBytes); + level1SerializedImpacts.length = numImpactBytes; + assert indexHasPos || docIn.getFilePointer() == skip1EndFP; + docIn.seek(skip1EndFP); + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + long skip0NumBytes = docIn.readVLong(); + // end offset of skip data (before the actual data starts) + long skip0End = docIn.getFilePointer() + skip0NumBytes; + int docDelta = readVInt15(docIn); + long blockLength = readVLong15(docIn); + + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + level0DocEndFP = docIn.getFilePointer() + blockLength; + int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes, 0, numImpactBytes); + level0SerializedImpacts.length = numImpactBytes; + docIn.seek(skip0End); + break; + } + + // skip block + docIn.skipBytes(blockLength); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > level0LastDocID) { // advance skip data on level 0 + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + skipLevel0To(target); + + needsRefilling = true; + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { + skipLevel1To(doc + 1); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + final long skip0Len = docIn.readVLong(); // skip len + final long skip0End = docIn.getFilePointer() + skip0Len; + final int docDelta = readVInt15(docIn); + final long blockLength = readVLong15(docIn); + level0LastDocID += docDelta; + level0DocEndFP = docIn.getFilePointer() + blockLength; + final int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes, 0, numImpactBytes); + level0SerializedImpacts.length = numImpactBytes; + docIn.seek(skip0End); + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + } + + refillDocs(); + needsRefilling = false; + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { + moveToNextLevel0Block(); + } else if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + return this.doc = (int) docBuffer[docBufferUpto++]; + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID || needsRefilling) { + advanceShallow(target); + refillDocs(); + needsRefilling = false; + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + this.doc = (int) docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public Impacts getImpacts() throws IOException { + return new Impacts() { + + @Override + public int numLevels() { + int numLevels = 0; + if (level0LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (level1LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (numLevels == 0) { + numLevels++; + } + return numLevels; + } + + @Override + public int getDocIdUpTo(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level0LastDocID; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level1LastDocID; + } + level--; + } + + return NO_MORE_DOCS; + } + + @Override + public List getImpacts(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level0SerializedImpactsIn.reset( + level0SerializedImpacts.bytes, 0, level0SerializedImpacts.length); + readImpacts(level0SerializedImpactsIn, level0Impacts); + return level0Impacts; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level1SerializedImpactsIn.reset( + level1SerializedImpacts.bytes, 0, level1SerializedImpacts.length); + readImpacts(level1SerializedImpactsIn, level1Impacts); + return level1Impacts; + } + level--; + } + + return Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)); + } + }; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsPostingsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + private int posBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final IndexInput posIn; + + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private long posPendingCount; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + // true if we shallow-advanced to a new block that we have not decoded yet + private boolean needsRefilling; + + // level 0 skip data + private int level0LastDocID; + private long level0DocEndFP; + private long level0PosEndFP; + private int level0BlockPosUpto; + private final BytesRefBuilder level0SerializedImpacts = new BytesRefBuilder(); + private final ByteArrayDataInput level0SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level0Impacts; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + private long level1PosEndFP; + private int level1BlockPosUpto; + private final BytesRefBuilder level1SerializedImpacts = new BytesRefBuilder(); + private final ByteArrayDataInput level1SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level1Impacts; + + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene912PostingsReader.this.posIn.clone(); + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + + docFreq = termState.docFreq; + posTermStartFP = termState.posStartFP; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + posIn.seek(posTermStartFP); + level1PosEndFP = posTermStartFP; + level0PosEndFP = posTermStartFP; + posPendingCount = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + doc = -1; + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + level1BlockPosUpto = 0; + docBufferUpto = BLOCK_SIZE; + posBufferUpto = BLOCK_SIZE; + level0SerializedImpacts.growNoCopy(maxImpactNumBytesAtLevel0); + level1SerializedImpacts.growNoCopy(maxImpactNumBytesAtLevel1); + level0Impacts = new MutableImpactList(maxNumImpactsAtLevel0); + level1Impacts = new MutableImpactList(maxNumImpactsAtLevel1); + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + level0PosEndFP = level1PosEndFP; + level0BlockPosUpto = level1BlockPosUpto; + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); + + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + int numImpactBytes = docIn.readShort(); + if (level1LastDocID >= target) { + docIn.readBytes(level1SerializedImpacts.bytes(), 0, numImpactBytes); + level1SerializedImpacts.setLength(numImpactBytes); + } else { + docIn.skipBytes(numImpactBytes); + } + level1PosEndFP += docIn.readVLong(); + level1BlockPosUpto = docIn.readByte(); + assert indexHasOffsetsOrPayloads || docIn.getFilePointer() == skip1EndFP; + + if (level1LastDocID >= target) { + docIn.seek(skip1EndFP); + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + + // If nextBlockPosFP is less than the current FP, it means that the block of positions for + // the first docs of the next block are already decoded. In this case we just accumulate + // frequencies into posPendingCount instead of seeking backwards and decoding the same pos + // block again. + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + posBufferUpto = BLOCK_SIZE; + } else { + for (int i = docBufferUpto; i < BLOCK_SIZE; ++i) { + posPendingCount += freqBuffer[i]; + } + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + long blockLength = readVLong15(docIn); + level0DocEndFP = docIn.getFilePointer() + blockLength; + + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes(), 0, numImpactBytes); + level0SerializedImpacts.setLength(numImpactBytes); + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + docIn.readVLong(); // pay fp delta + docIn.readVInt(); // pay upto + } + break; + } + // skip block + docIn.skipBytes(docIn.readVLong()); // impacts + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readVInt(); + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > level0LastDocID) { // advance level 0 skip data + + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + skipLevel0To(target); + + needsRefilling = true; + } + } + + @Override + public Impacts getImpacts() throws IOException { + return new Impacts() { + + @Override + public int numLevels() { + int numLevels = 0; + if (level0LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (level1LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (numLevels == 0) { + numLevels++; + } + return numLevels; + } + + @Override + public int getDocIdUpTo(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level0LastDocID; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level1LastDocID; + } + level--; + } + + return NO_MORE_DOCS; + } + + @Override + public List getImpacts(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level0SerializedImpactsIn.reset( + level0SerializedImpacts.bytes(), 0, level0SerializedImpacts.length()); + readImpacts(level0SerializedImpactsIn, level0Impacts); + return level0Impacts; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level1SerializedImpactsIn.reset( + level1SerializedImpacts.bytes(), 0, level1SerializedImpacts.length()); + readImpacts(level1SerializedImpactsIn, level1Impacts); + return level1Impacts; + } + level--; + } + + return Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)); + } + }; + } + + @Override + public int nextDoc() throws IOException { + advanceShallow(doc + 1); + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + doc = (int) docBuffer[docBufferUpto]; + freq = (int) freqBuffer[docBufferUpto]; + posPendingCount += freq; + docBufferUpto++; + position = 0; + return this.doc; + } + + @Override + public int advance(int target) throws IOException { + advanceShallow(target); + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + for (int i = docBufferUpto; i <= next; ++i) { + posPendingCount += freqBuffer[i]; + } + freq = (int) freqBuffer[next]; + docBufferUpto = next + 1; + position = 0; + return this.doc = (int) docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + long toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + posBufferUpto += toSkip; + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + pforUtil.skip(posIn); + toSkip -= BLOCK_SIZE; + } + refillPositions(); + posBufferUpto = (int) toSkip; + } + + position = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + posIn.skipBytes(payloadLength); + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + posIn.readVInt(); // offset length + } + } + } + } else { + pforUtil.decode(posIn, posDeltaBuffer); + } + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + posBufferUpto++; + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + @Override + public long cost() { + return docFreq; + } + } + + /** + * @see Lucene912PostingsWriter#writeVInt15(org.apache.lucene.store.DataOutput, int) + */ + static int readVInt15(DataInput in) throws IOException { + short s = in.readShort(); + if (s >= 0) { + return s; + } else { + return (s & 0x7FFF) | (in.readVInt() << 15); + } + } + + /** + * @see Lucene912PostingsWriter#writeVLong15(org.apache.lucene.store.DataOutput, long) + */ + static long readVLong15(DataInput in) throws IOException { + short s = in.readShort(); + if (s >= 0) { + return s; + } else { + return (s & 0x7FFFL) | (in.readVLong() << 15); + } + } + + private void prefetchPostings(IndexInput docIn, IntBlockTermState state) throws IOException { + if (state.docFreq > 1 && docIn.getFilePointer() != state.docStartFP) { + // Don't prefetch if the input is already positioned at the right offset, which suggests that + // the caller is streaming the entire inverted index (e.g. for merging), let the read-ahead + // logic do its work instead. Note that this heuristic doesn't work for terms that have skip + // data, since skip data is stored after the last term, but handling all terms that have <128 + // docs is a good start already. + docIn.prefetch(state.docStartFP, 1); + } + // Note: we don't prefetch positions or offsets, which are less likely to be needed. + } + + static class MutableImpactList extends AbstractList implements RandomAccess { + int length; + final Impact[] impacts; + + MutableImpactList(int capacity) { + impacts = new Impact[capacity]; + for (int i = 0; i < capacity; ++i) { + impacts[i] = new Impact(Integer.MAX_VALUE, 1L); + } + } + + @Override + public Impact get(int index) { + return impacts[index]; + } + + @Override + public int size() { + return length; + } + } + + static MutableImpactList readImpacts(ByteArrayDataInput in, MutableImpactList reuse) { + int freq = 0; + long norm = 0; + int length = 0; + while (in.getPosition() < in.length()) { + int freqDelta = in.readVInt(); + if ((freqDelta & 0x01) != 0) { + freq += 1 + (freqDelta >>> 1); + try { + norm += 1 + in.readZLong(); + } catch (IOException e) { + throw new RuntimeException(e); // cannot happen on a BADI + } + } else { + freq += 1 + (freqDelta >>> 1); + norm++; + } + Impact impact = reuse.impacts[length]; + impact.freq = freq; + impact.norm = norm; + length++; + } + reuse.length = length; + return reuse; + } + + @Override + public void checkIntegrity() throws IOException { + if (docIn != null) { + CodecUtil.checksumEntireFile(docIn); + } + if (posIn != null) { + CodecUtil.checksumEntireFile(posIn); + } + if (payIn != null) { + CodecUtil.checksumEntireFile(payIn); + } + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "(positions=" + + (posIn != null) + + ",payloads=" + + (payIn != null) + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java new file mode 100644 index 00000000000..b307080b215 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java @@ -0,0 +1,681 @@ +/* + * 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.codecs.lucene912; + +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.*; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.CompetitiveImpactAccumulator; +import org.apache.lucene.codecs.PushPostingsWriterBase; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; + +/** Writer for {@link Lucene912PostingsFormat}. */ +public class Lucene912PostingsWriter extends PushPostingsWriterBase { + + static final IntBlockTermState EMPTY_STATE = new IntBlockTermState(); + + IndexOutput metaOut; + IndexOutput docOut; + IndexOutput posOut; + IndexOutput payOut; + + IntBlockTermState lastState; + + // Holds starting file pointers for current term: + private long docStartFP; + private long posStartFP; + private long payStartFP; + + final long[] docDeltaBuffer; + final long[] freqBuffer; + private int docBufferUpto; + + final long[] posDeltaBuffer; + final long[] payloadLengthBuffer; + final long[] offsetStartDeltaBuffer; + final long[] offsetLengthBuffer; + private int posBufferUpto; + + private byte[] payloadBytes; + private int payloadByteUpto; + + private int level0LastDocID; + private long level0LastPosFP; + private long level0LastPayFP; + + private int level1LastDocID; + private long level1LastPosFP; + private long level1LastPayFP; + + private int docID; + private int lastDocID; + private int lastPosition; + private int lastStartOffset; + private int docCount; + + private final PForUtil pforUtil; + private final ForDeltaUtil forDeltaUtil; + + private boolean fieldHasNorms; + private NumericDocValues norms; + private final CompetitiveImpactAccumulator level0FreqNormAccumulator = + new CompetitiveImpactAccumulator(); + private final CompetitiveImpactAccumulator level1CompetitiveFreqNormAccumulator = + new CompetitiveImpactAccumulator(); + + private int maxNumImpactsAtLevel0; + private int maxImpactNumBytesAtLevel0; + private int maxNumImpactsAtLevel1; + private int maxImpactNumBytesAtLevel1; + + /** Scratch output that we use to be able to prepend the encoded length, e.g. impacts. */ + private final ByteBuffersDataOutput scratchOutput = ByteBuffersDataOutput.newResettableInstance(); + + /** + * Output for a single block. This is useful to be able to prepend skip data before each block, + * which can only be computed once the block is encoded. The content is then typically copied to + * {@link #level1Output}. + */ + private final ByteBuffersDataOutput level0Output = ByteBuffersDataOutput.newResettableInstance(); + + /** + * Output for groups of 32 blocks. This is useful to prepend skip data for these 32 blocks, which + * can only be done once we have encoded these 32 blocks. The content is then typically copied to + * {@link #docCount}. + */ + private final ByteBuffersDataOutput level1Output = ByteBuffersDataOutput.newResettableInstance(); + + /** Sole constructor. */ + public Lucene912PostingsWriter(SegmentWriteState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.META_EXTENSION); + String docFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.DOC_EXTENSION); + metaOut = state.directory.createOutput(metaFileName, state.context); + IndexOutput posOut = null; + IndexOutput payOut = null; + boolean success = false; + try { + docOut = state.directory.createOutput(docFileName, state.context); + CodecUtil.writeIndexHeader( + metaOut, META_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.writeIndexHeader( + docOut, DOC_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + final ForUtil forUtil = new ForUtil(); + forDeltaUtil = new ForDeltaUtil(forUtil); + pforUtil = new PForUtil(forUtil); + if (state.fieldInfos.hasProx()) { + posDeltaBuffer = new long[BLOCK_SIZE]; + String posFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.POS_EXTENSION); + posOut = state.directory.createOutput(posFileName, state.context); + CodecUtil.writeIndexHeader( + posOut, POS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + + if (state.fieldInfos.hasPayloads()) { + payloadBytes = new byte[128]; + payloadLengthBuffer = new long[BLOCK_SIZE]; + } else { + payloadBytes = null; + payloadLengthBuffer = null; + } + + if (state.fieldInfos.hasOffsets()) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + } + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene912PostingsFormat.PAY_EXTENSION); + payOut = state.directory.createOutput(payFileName, state.context); + CodecUtil.writeIndexHeader( + payOut, PAY_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + } + } else { + posDeltaBuffer = null; + payloadLengthBuffer = null; + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + payloadBytes = null; + } + this.payOut = payOut; + this.posOut = posOut; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut); + } + } + + docDeltaBuffer = new long[BLOCK_SIZE]; + freqBuffer = new long[BLOCK_SIZE]; + } + + @Override + public IntBlockTermState newTermState() { + return new IntBlockTermState(); + } + + @Override + public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException { + CodecUtil.writeIndexHeader( + termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + termsOut.writeVInt(BLOCK_SIZE); + } + + @Override + public void setField(FieldInfo fieldInfo) { + super.setField(fieldInfo); + lastState = EMPTY_STATE; + fieldHasNorms = fieldInfo.hasNorms(); + } + + @Override + public void startTerm(NumericDocValues norms) { + docStartFP = docOut.getFilePointer(); + if (writePositions) { + posStartFP = posOut.getFilePointer(); + level1LastPosFP = level0LastPosFP = posStartFP; + if (writePayloads || writeOffsets) { + payStartFP = payOut.getFilePointer(); + level1LastPayFP = level0LastPayFP = payStartFP; + } + } + lastDocID = -1; + level0LastDocID = -1; + level1LastDocID = -1; + this.norms = norms; + if (writeFreqs) { + level0FreqNormAccumulator.clear(); + } + } + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + flushDocBlock(false); + docBufferUpto = 0; + } + + final int docDelta = docID - lastDocID; + + if (docID < 0 || docDelta <= 0) { + throw new CorruptIndexException( + "docs out of order (" + docID + " <= " + lastDocID + " )", docOut); + } + + docDeltaBuffer[docBufferUpto] = docDelta; + if (writeFreqs) { + freqBuffer[docBufferUpto] = termDocFreq; + } + + this.docID = docID; + lastPosition = 0; + lastStartOffset = 0; + + if (writeFreqs) { + long norm; + if (fieldHasNorms) { + boolean found = norms.advanceExact(docID); + if (found == false) { + // This can happen if indexing hits a problem after adding a doc to the + // postings but before buffering the norm. Such documents are written + // deleted and will go away on the first merge. + norm = 1L; + } else { + norm = norms.longValue(); + assert norm != 0 : docID; + } + } else { + norm = 1L; + } + + level0FreqNormAccumulator.add(termDocFreq, norm); + } + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) + throws IOException { + if (position > IndexWriter.MAX_POSITION) { + throw new CorruptIndexException( + "position=" + + position + + " is too large (> IndexWriter.MAX_POSITION=" + + IndexWriter.MAX_POSITION + + ")", + docOut); + } + if (position < 0) { + throw new CorruptIndexException("position=" + position + " is < 0", docOut); + } + posDeltaBuffer[posBufferUpto] = position - lastPosition; + if (writePayloads) { + if (payload == null || payload.length == 0) { + // no payload + payloadLengthBuffer[posBufferUpto] = 0; + } else { + payloadLengthBuffer[posBufferUpto] = payload.length; + if (payloadByteUpto + payload.length > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length); + } + System.arraycopy( + payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length); + payloadByteUpto += payload.length; + } + } + + if (writeOffsets) { + assert startOffset >= lastStartOffset; + assert endOffset >= startOffset; + offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset; + offsetLengthBuffer[posBufferUpto] = endOffset - startOffset; + lastStartOffset = startOffset; + } + + posBufferUpto++; + lastPosition = position; + if (posBufferUpto == BLOCK_SIZE) { + pforUtil.encode(posDeltaBuffer, posOut); + + if (writePayloads) { + pforUtil.encode(payloadLengthBuffer, payOut); + payOut.writeVInt(payloadByteUpto); + payOut.writeBytes(payloadBytes, 0, payloadByteUpto); + payloadByteUpto = 0; + } + if (writeOffsets) { + pforUtil.encode(offsetStartDeltaBuffer, payOut); + pforUtil.encode(offsetLengthBuffer, payOut); + } + posBufferUpto = 0; + } + } + + @Override + public void finishDoc() throws IOException { + docBufferUpto++; + docCount++; + + lastDocID = docID; + } + + /** + * Special vints that are encoded on 2 bytes if they require 15 bits or less. VInt becomes + * especially slow when the number of bytes is variable, so this special layout helps in the case + * when the number likely requires 15 bits or less + */ + static void writeVInt15(DataOutput out, int v) throws IOException { + assert v >= 0; + writeVLong15(out, v); + } + + /** + * @see #writeVInt15(DataOutput, int) + */ + static void writeVLong15(DataOutput out, long v) throws IOException { + assert v >= 0; + if ((v & ~0x7FFFL) == 0) { + out.writeShort((short) v); + } else { + out.writeShort((short) (0x8000 | (v & 0x7FFF))); + out.writeVLong(v >> 15); + } + } + + private void flushDocBlock(boolean finishTerm) throws IOException { + assert docBufferUpto != 0; + + if (docBufferUpto < BLOCK_SIZE) { + assert finishTerm; + PostingsUtil.writeVIntBlock( + level0Output, docDeltaBuffer, freqBuffer, docBufferUpto, writeFreqs); + } else { + if (writeFreqs) { + List impacts = level0FreqNormAccumulator.getCompetitiveFreqNormPairs(); + if (impacts.size() > maxNumImpactsAtLevel0) { + maxNumImpactsAtLevel0 = impacts.size(); + } + writeImpacts(impacts, scratchOutput); + assert level0Output.size() == 0; + if (scratchOutput.size() > maxImpactNumBytesAtLevel0) { + maxImpactNumBytesAtLevel0 = Math.toIntExact(scratchOutput.size()); + } + level0Output.writeVLong(scratchOutput.size()); + scratchOutput.copyTo(level0Output); + scratchOutput.reset(); + if (writePositions) { + level0Output.writeVLong(posOut.getFilePointer() - level0LastPosFP); + level0Output.writeByte((byte) posBufferUpto); + level0LastPosFP = posOut.getFilePointer(); + + if (writeOffsets || writePayloads) { + level0Output.writeVLong(payOut.getFilePointer() - level0LastPayFP); + level0Output.writeVInt(payloadByteUpto); + level0LastPayFP = payOut.getFilePointer(); + } + } + } + long numSkipBytes = level0Output.size(); + forDeltaUtil.encodeDeltas(docDeltaBuffer, level0Output); + if (writeFreqs) { + pforUtil.encode(freqBuffer, level0Output); + } + + // docID - lastBlockDocID is at least 128, so it can never fit a single byte with a vint + // Even if we subtracted 128, only extremely dense blocks would be eligible to a single byte + // so let's go with 2 bytes right away + writeVInt15(scratchOutput, docID - level0LastDocID); + writeVLong15(scratchOutput, level0Output.size()); + numSkipBytes += scratchOutput.size(); + level1Output.writeVLong(numSkipBytes); + scratchOutput.copyTo(level1Output); + scratchOutput.reset(); + } + + level0Output.copyTo(level1Output); + level0Output.reset(); + level0LastDocID = docID; + if (writeFreqs) { + level1CompetitiveFreqNormAccumulator.addAll(level0FreqNormAccumulator); + level0FreqNormAccumulator.clear(); + } + + if ((docCount & LEVEL1_MASK) == 0) { // true every 32 blocks (4,096 docs) + writeLevel1SkipData(); + level1LastDocID = docID; + level1CompetitiveFreqNormAccumulator.clear(); + } else if (finishTerm) { + level1Output.copyTo(docOut); + level1Output.reset(); + level1CompetitiveFreqNormAccumulator.clear(); + } + } + + private void writeLevel1SkipData() throws IOException { + docOut.writeVInt(docID - level1LastDocID); + long numImpactBytes = scratchOutput.size(); + final long level1End; + if (writeFreqs) { + List impacts = level1CompetitiveFreqNormAccumulator.getCompetitiveFreqNormPairs(); + if (impacts.size() > maxNumImpactsAtLevel1) { + maxNumImpactsAtLevel1 = impacts.size(); + } + writeImpacts(impacts, scratchOutput); + numImpactBytes = scratchOutput.size(); + if (numImpactBytes > maxImpactNumBytesAtLevel1) { + maxImpactNumBytesAtLevel1 = Math.toIntExact(numImpactBytes); + } + if (writePositions) { + scratchOutput.writeVLong(posOut.getFilePointer() - level1LastPosFP); + scratchOutput.writeByte((byte) posBufferUpto); + level1LastPosFP = posOut.getFilePointer(); + if (writeOffsets || writePayloads) { + scratchOutput.writeVLong(payOut.getFilePointer() - level1LastPayFP); + scratchOutput.writeVInt(payloadByteUpto); + level1LastPayFP = payOut.getFilePointer(); + } + } + final long level1Len = 2 * Short.BYTES + scratchOutput.size() + level1Output.size(); + docOut.writeVLong(level1Len); + level1End = docOut.getFilePointer() + level1Len; + // There are at most 128 impacts, that require at most 2 bytes each + assert numImpactBytes <= Short.MAX_VALUE; + // Like impacts plus a few vlongs, still way under the max short value + assert scratchOutput.size() + Short.BYTES <= Short.MAX_VALUE; + docOut.writeShort((short) (scratchOutput.size() + Short.BYTES)); + docOut.writeShort((short) numImpactBytes); + scratchOutput.copyTo(docOut); + scratchOutput.reset(); + } else { + docOut.writeVLong(level1Output.size()); + level1End = docOut.getFilePointer() + level1Output.size(); + } + level1Output.copyTo(docOut); + level1Output.reset(); + assert docOut.getFilePointer() == level1End : docOut.getFilePointer() + " " + level1End; + } + + static void writeImpacts(Collection impacts, DataOutput out) throws IOException { + Impact previous = new Impact(0, 0); + for (Impact impact : impacts) { + assert impact.freq > previous.freq; + assert Long.compareUnsigned(impact.norm, previous.norm) > 0; + int freqDelta = impact.freq - previous.freq - 1; + long normDelta = impact.norm - previous.norm - 1; + if (normDelta == 0) { + // most of time, norm only increases by 1, so we can fold everything in a single byte + out.writeVInt(freqDelta << 1); + } else { + out.writeVInt((freqDelta << 1) | 1); + out.writeZLong(normDelta); + } + previous = impact; + } + } + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(BlockTermState _state) throws IOException { + IntBlockTermState state = (IntBlockTermState) _state; + assert state.docFreq > 0; + + // TODO: wasteful we are counting this (counting # docs + // for this term) in two places? + assert state.docFreq == docCount : state.docFreq + " vs " + docCount; + + // docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to + // it. + final int singletonDocID; + if (state.docFreq == 1) { + // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq + singletonDocID = (int) docDeltaBuffer[0] - 1; + } else { + singletonDocID = -1; + flushDocBlock(true); + } + + final long lastPosBlockOffset; + + if (writePositions) { + // totalTermFreq is just total number of positions(or payloads, or offsets) + // associated with current term. + assert state.totalTermFreq != -1; + if (state.totalTermFreq > BLOCK_SIZE) { + // record file offset for last pos in last block + lastPosBlockOffset = posOut.getFilePointer() - posStartFP; + } else { + lastPosBlockOffset = -1; + } + if (posBufferUpto > 0) { + assert posBufferUpto < BLOCK_SIZE; + // TODO: should we send offsets/payloads to + // .pay...? seems wasteful (have to store extra + // vLong for low (< BLOCK_SIZE) DF terms = vast vast + // majority) + + // vInt encode the remaining positions/payloads/offsets: + int lastPayloadLength = -1; // force first payload length to be written + int lastOffsetLength = -1; // force first offset length to be written + int payloadBytesReadUpto = 0; + for (int i = 0; i < posBufferUpto; i++) { + final int posDelta = (int) posDeltaBuffer[i]; + if (writePayloads) { + final int payloadLength = (int) payloadLengthBuffer[i]; + if (payloadLength != lastPayloadLength) { + lastPayloadLength = payloadLength; + posOut.writeVInt((posDelta << 1) | 1); + posOut.writeVInt(payloadLength); + } else { + posOut.writeVInt(posDelta << 1); + } + + if (payloadLength != 0) { + posOut.writeBytes(payloadBytes, payloadBytesReadUpto, payloadLength); + payloadBytesReadUpto += payloadLength; + } + } else { + posOut.writeVInt(posDelta); + } + + if (writeOffsets) { + int delta = (int) offsetStartDeltaBuffer[i]; + int length = (int) offsetLengthBuffer[i]; + if (length == lastOffsetLength) { + posOut.writeVInt(delta << 1); + } else { + posOut.writeVInt(delta << 1 | 1); + posOut.writeVInt(length); + lastOffsetLength = length; + } + } + } + + if (writePayloads) { + assert payloadBytesReadUpto == payloadByteUpto; + payloadByteUpto = 0; + } + } + } else { + lastPosBlockOffset = -1; + } + + state.docStartFP = docStartFP; + state.posStartFP = posStartFP; + state.payStartFP = payStartFP; + state.singletonDocID = singletonDocID; + + state.lastPosBlockOffset = lastPosBlockOffset; + docBufferUpto = 0; + posBufferUpto = 0; + lastDocID = -1; + docCount = 0; + } + + @Override + public void encodeTerm( + DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) + throws IOException { + IntBlockTermState state = (IntBlockTermState) _state; + if (absolute) { + lastState = EMPTY_STATE; + assert lastState.docStartFP == 0; + } + + if (lastState.singletonDocID != -1 + && state.singletonDocID != -1 + && state.docStartFP == lastState.docStartFP) { + // With runs of rare values such as ID fields, the increment of pointers in the docs file is + // often 0. + // Furthermore some ID schemes like auto-increment IDs or Flake IDs are monotonic, so we + // encode the delta + // between consecutive doc IDs to save space. + final long delta = (long) state.singletonDocID - lastState.singletonDocID; + out.writeVLong((BitUtil.zigZagEncode(delta) << 1) | 0x01); + } else { + out.writeVLong((state.docStartFP - lastState.docStartFP) << 1); + if (state.singletonDocID != -1) { + out.writeVInt(state.singletonDocID); + } + } + + if (writePositions) { + out.writeVLong(state.posStartFP - lastState.posStartFP); + if (writePayloads || writeOffsets) { + out.writeVLong(state.payStartFP - lastState.payStartFP); + } + } + if (writePositions) { + if (state.lastPosBlockOffset != -1) { + out.writeVLong(state.lastPosBlockOffset); + } + } + lastState = state; + } + + @Override + public void close() throws IOException { + // TODO: add a finish() at least to PushBase? DV too...? + boolean success = false; + try { + if (docOut != null) { + CodecUtil.writeFooter(docOut); + } + if (posOut != null) { + CodecUtil.writeFooter(posOut); + } + if (payOut != null) { + CodecUtil.writeFooter(payOut); + } + if (metaOut != null) { + metaOut.writeInt(maxNumImpactsAtLevel0); + metaOut.writeInt(maxImpactNumBytesAtLevel0); + metaOut.writeInt(maxNumImpactsAtLevel1); + metaOut.writeInt(maxImpactNumBytesAtLevel1); + metaOut.writeLong(docOut.getFilePointer()); + if (posOut != null) { + metaOut.writeLong(posOut.getFilePointer()); + if (payOut != null) { + metaOut.writeLong(payOut.getFilePointer()); + } + } + CodecUtil.writeFooter(metaOut); + } + success = true; + } finally { + if (success) { + IOUtils.close(metaOut, docOut, posOut, payOut); + } else { + IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut); + } + metaOut = docOut = posOut = payOut = null; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/PForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java index de62fb6b40b..f4405ae66fa 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import java.util.Arrays; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java new file mode 100644 index 00000000000..4834dd73e22 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java @@ -0,0 +1,73 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; + +/** Utility class to encode/decode postings block. */ +final class PostingsUtil { + + /** + * Read values that have been written using variable-length encoding and group-varint encoding + * instead of bit-packing. + */ + static void readVIntBlock( + IndexInput docIn, + long[] docBuffer, + long[] freqBuffer, + int num, + boolean indexHasFreq, + boolean decodeFreq) + throws IOException { + docIn.readGroupVInts(docBuffer, num); + if (indexHasFreq && decodeFreq) { + for (int i = 0; i < num; ++i) { + freqBuffer[i] = docBuffer[i] & 0x01; + docBuffer[i] >>= 1; + if (freqBuffer[i] == 0) { + freqBuffer[i] = docIn.readVInt(); + } + } + } else if (indexHasFreq) { + for (int i = 0; i < num; ++i) { + docBuffer[i] >>= 1; + } + } + } + + /** Write freq buffer with variable-length encoding and doc buffer with group-varint encoding. */ + static void writeVIntBlock( + DataOutput docOut, long[] docBuffer, long[] freqBuffer, int num, boolean writeFreqs) + throws IOException { + if (writeFreqs) { + for (int i = 0; i < num; i++) { + docBuffer[i] = (docBuffer[i] << 1) | (freqBuffer[i] == 1 ? 1 : 0); + } + } + docOut.writeGroupVInts(docBuffer, num); + if (writeFreqs) { + for (int i = 0; i < num; i++) { + final int freq = (int) freqBuffer[i]; + if (freq != 1) { + docOut.writeVInt(freq); + } + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py index cd81d493570..c6a33ceef53 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py @@ -40,7 +40,7 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java new file mode 100644 index 00000000000..853f86a855a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java @@ -0,0 +1,431 @@ +/* + * 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. + */ + +/** + * Lucene 9.12 file format. + * + *

    Apache Lucene - Index File Formats

    + * + * + * + *

    Introduction

    + * + *
    + * + *

    This document defines the index file formats used in this version of Lucene. If you are using + * a different version of Lucene, please consult the copy of docs/ that was distributed + * with the version you are using. + * + *

    This document attempts to provide a high-level definition of the Apache Lucene file formats. + *

    + * + *

    Definitions

    + * + *
    + * + *

    The fundamental concepts in Lucene are index, document, field and term. + * + *

    An index contains a sequence of documents. + * + *

      + *
    • A document is a sequence of fields. + *
    • A field is a named sequence of terms. + *
    • A term is a sequence of bytes. + *
    + * + *

    The same sequence of bytes in two different fields is considered a different term. Thus terms + * are represented as a pair: the string naming the field, and the bytes within the field. + * + *

    Inverted Indexing

    + * + *

    Lucene's index stores terms and statistics about those terms in order to make term-based + * search more efficient. Lucene's terms index falls into the family of indexes known as an + * inverted index. This is because it can list, for a term, the documents that contain it. + * This is the inverse of the natural relationship, in which documents list terms. + * + *

    Types of Fields

    + * + *

    In Lucene, fields may be stored, in which case their text is stored in the index + * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field + * may be both stored and indexed. + * + *

    The text of a field may be tokenized into terms to be indexed, or the text of a field + * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is + * useful for certain identifier fields to be indexed literally. + * + *

    See the {@link org.apache.lucene.document.Field Field} java docs for more information on + * Fields. + * + *

    Segments

    + * + *

    Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a + * fully independent index, which could be searched separately. Indexes evolve by: + * + *

      + *
    1. Creating new segments for newly added documents. + *
    2. Merging existing segments. + *
    + * + *

    Searches may involve multiple segments and/or multiple indexes, each index potentially + * composed of a set of segments. + * + *

    Document Numbers

    + * + *

    Internally, Lucene refers to documents by an integer document number. The first + * document added to an index is numbered zero, and each subsequent document added gets a number one + * greater than the previous. + * + *

    Note that a document's number may change, so caution should be taken when storing these + * numbers outside of Lucene. In particular, numbers may change in the following situations: + * + *

      + *
    • + *

      The numbers stored in each segment are unique only within the segment, and must be + * converted before they can be used in a larger context. The standard technique is to + * allocate each segment a range of values, based on the range of numbers used in that + * segment. To convert a document number from a segment to an external value, the segment's + * base document number is added. To convert an external value back to a + * segment-specific value, the segment is identified by the range that the external value is + * in, and the segment's base value is subtracted. For example two five document segments + * might be combined, so that the first segment has a base value of zero, and the second of + * five. Document three from the second segment would have an external value of eight. + *

    • + *

      When documents are deleted, gaps are created in the numbering. These are eventually + * removed as the index evolves through merging. Deleted documents are dropped when segments + * are merged. A freshly-merged segment thus has no gaps in its numbering. + *

    + * + *
    + * + *

    Index Structure Overview

    + * + *
    + * + *

    Each segment index maintains the following: + * + *

      + *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This + * contains metadata about a segment, such as the number of documents, what files it uses, and + * information about how the segment is sorted + *
    • {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This + * contains metadata about the set of named fields used in the index. + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}. + * This contains, for each document, a list of attribute-value pairs, where the attributes are + * field names. These are used to store auxiliary information about the document, such as its + * title, url, or an identifier to access a database. The set of stored fields are what is + * returned for each hit when searching. This is keyed by document number. + *
    • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term dictionary}. A + * dictionary containing all of the terms used in all of the indexed fields of all of the + * documents. The dictionary also contains the number of documents which contain the term, and + * pointers to the term's frequency and proximity data. + *
    • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Frequency data}. For + * each term in the dictionary, the numbers of all the documents that contain that term, and + * the frequency of the term in that document, unless frequencies are omitted ({@link + * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) + *
    • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Proximity data}. For + * each term in the dictionary, the positions that the term occurs in each document. Note that + * this will not exist if all fields in all documents omit position data. + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For + * each field in each document, a value is stored that is multiplied into the score for hits + * on that field. + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each + * field in each document, the term vector (sometimes called document vector) may be stored. A + * term vector consists of term text and term frequency. To add Term Vectors to your index see + * the {@link org.apache.lucene.document.Field Field} constructors + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like + * stored values, these are also keyed by document number, but are generally intended to be + * loaded into main memory for fast access. Whereas stored values are generally intended for + * summary results from searches, per-document values are useful for things like scoring + * factors. + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An + * optional file indicating which documents are live. + *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair + * of files, recording dimensionally indexed fields, to enable fast numeric range filtering + * and large numeric values like BigInteger and BigDecimal (1D) and geographic shape + * intersection (2D, 3D). + *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The + * vector format stores numeric vectors in a format optimized for random access and + * computation, supporting high-dimensional nearest-neighbor search. + *
    + * + *

    Details on each of these are provided in their linked pages.

    + * + *

    File Naming

    + * + *
    + * + *

    All files belonging to a segment have the same name with varying extensions. The extensions + * correspond to the different file formats described below. When using the Compound File format + * (default for small segments) these files (except for the Segment info file, the Lock file, and + * Deleted documents file) are collapsed into a single .cfs file (see below for details) + * + *

    Typically, all segments in an index are stored in a single directory, although this is not + * required. + * + *

    File names are never re-used. That is, when any file is saved to the Directory it is given a + * never before used filename. This is achieved using a simple generations approach. For example, + * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long + * integer represented in alpha-numeric (base 36) form.

    + * + *

    Summary of File Extensions

    + * + *
    + * + *

    The following table summarizes the names and extensions of the files in Lucene: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
    lucene filenames by extension
    NameExtensionBrief Description
    {@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
    Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same + * file.
    {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}.siStores metadata about a segment
    {@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for + * systems that frequently run out of file handles.
    {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}.fnmStores information about the fields
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
    {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
    {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Index}.tipThe index into the Term Dictionary
    {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
    {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Positions}.posStores position information about where a term occurs in the index
    {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
    {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
    {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
    {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
    {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
    {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
    {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.dii, .dimHolds indexed points
    {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}.vec, .vem, .veq, vexHolds indexed vectors; .vec files contain the raw vector data, + * .vem the vector metadata, .veq the quantized vector data, and .vex the + * hnsw graph data.
    + * + *

    + * + *

    Lock File

    + * + * The write lock, which is stored in the index directory by default, is named "write.lock". If the + * lock directory is different from the index directory then the write lock will be named + * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index + * directory. When this file is present, a writer is currently modifying the index (adding or + * removing documents). This lock file ensures that only one writer is modifying the index at a + * time. + * + *

    History

    + * + *

    Compatibility notes are provided in this document, describing how file formats have changed + * from prior versions: + * + *

      + *
    • In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit + * lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching + * or adding/deleting of docs. When the new segments file is saved (committed), it will be + * written in the new file format (meaning no specific "upgrade" process is needed). But note + * that once a commit has occurred, pre-2.1 Lucene will not be able to read the index. + *
    • In version 2.3, the file format was changed to allow segments to share a single set of doc + * store (vectors & stored fields) files. This allows for faster indexing in certain + * cases. The change is fully backwards compatible (in the same way as the lock-less commits + * change in 2.1). + *
    • In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified + * UTF-8. See LUCENE-510 for + * details. + *
    • In version 2.9, an optional opaque Map<String,String> CommitUserData may be passed to + * IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N + * file. See LUCENE-1382 for + * details. Also, diagnostics were added to each segment written recording details about why + * it was written (due to flush, merge; which OS/JRE was used; etc.). See issue LUCENE-1654 for details. + *
    • In version 3.0, compressed fields are no longer written to the index (they can still be + * read, but on merge the new segment will write them, uncompressed). See issue LUCENE-1960 for details. + *
    • In version 3.1, segments records the code version that created them. See LUCENE-2720 for details. + * Additionally segments track explicitly whether or not they have term vectors. See LUCENE-2811 for details. + *
    • In version 3.2, numeric fields are written as natively to stored fields file, previously + * they were stored in text format only. + *
    • In version 3.4, fields can omit position data while still indexing term frequencies. + *
    • In version 4.0, the format of the inverted index became extensible via the {@link + * org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues}) + * was introduced. Normalization factors need no longer be a single byte, they can be any + * {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be + * unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into + * the postings lists. Payloads can be stored in the term vectors. + *
    • In version 4.1, the format of the postings list changed to use either of FOR compression or + * variable-byte encoding, depending upon the frequency of the term. Terms appearing only once + * were changed to inline directly into the term dictionary. Stored fields are compressed by + * default. + *
    • In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued + * type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields. + *
    • In version 4.5, DocValues were extended to explicitly represent missing values. + *
    • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to + * allow updating NumericDocValues fields. + *
    • In version 4.8, checksum footers were added to the end of each index file for improved data + * integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32 + * checksum of the file. + *
    • In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is + * suitable for faceting/sorting/analytics. + *
    • In version 5.4, DocValues have been improved to store more information on disk: addresses + * for binary fields and ord indexes for multi-valued fields. + *
    • In version 6.0, Points were added, for multi-dimensional range/distance search. + *
    • In version 6.2, new Segment info format that reads/writes the index sort, to support index + * sorting. + *
    • In version 7.0, DocValues have been improved to better support sparse doc values thanks to + * an iterator API. + *
    • In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term + * freq, normalization factor) pairs that may trigger the maximum score of the block. This + * information is recorded alongside skip data in order to be able to skip blocks of doc ids + * if they may not produce high enough scores. Additionally doc values and norms has been + * extended with jump-tables to make access O(1) instead of O(n), where n is the number of + * elements to skip when advancing in the data. + *
    • In version 8.4, postings, positions, offsets and payload lengths have move to a more + * performant encoding that is vectorized. + *
    • In version 8.6, index sort serialization is delegated to the sorts themselves, to allow + * user-defined sorts to be used + *
    • In version 8.7, stored fields compression became adaptive to better handle documents with + * smaller stored fields. + *
    • In version 9.0, vector-valued fields were added. + *
    • In version 9.1, vector-valued fields were modified to add a graph hierarchy. + *
    • In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by + * IndexDISI. ordToDoc mappings was added to .vem. + *
    • In version 9.5, HNSW graph connections were changed to be delta-encoded with vints. + * Additionally, metadata file size improvements were made by delta-encoding nodes by graph + * layer and not writing the node ids for the zeroth layer. + *
    • In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector + * format to utilize int8 quantized vectors for float32 vector search. + *
    • In version 9.12, skip data was refactored to have only two levels: every 128 docs and every + * 4,06 docs, and to be inlined in postings lists. This resulted in a speedup for queries that + * need skipping, especially conjunctions. + *
    + * + * + * + *

    Limitations

    + * + *
    + * + *

    Lucene uses a Java int to refer to document numbers, and the index file format + * uses an Int32 on-disk to store document numbers. This is a limitation of both the + * index file format and the current implementation. Eventually these should be replaced with either + * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt + * VInt} values which have no limit.

    + */ +package org.apache.lucene.codecs.lucene912; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java index 2df2cfc9a30..98a029853e7 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java @@ -15,414 +15,5 @@ * limitations under the License. */ -/** - * Lucene 9.9 file format. - * - *

    Apache Lucene - Index File Formats

    - * - * - * - *

    Introduction

    - * - *
    - * - *

    This document defines the index file formats used in this version of Lucene. If you are using - * a different version of Lucene, please consult the copy of docs/ that was distributed - * with the version you are using. - * - *

    This document attempts to provide a high-level definition of the Apache Lucene file formats. - *

    - * - *

    Definitions

    - * - *
    - * - *

    The fundamental concepts in Lucene are index, document, field and term. - * - *

    An index contains a sequence of documents. - * - *

      - *
    • A document is a sequence of fields. - *
    • A field is a named sequence of terms. - *
    • A term is a sequence of bytes. - *
    - * - *

    The same sequence of bytes in two different fields is considered a different term. Thus terms - * are represented as a pair: the string naming the field, and the bytes within the field. - * - *

    Inverted Indexing

    - * - *

    Lucene's index stores terms and statistics about those terms in order to make term-based - * search more efficient. Lucene's terms index falls into the family of indexes known as an - * inverted index. This is because it can list, for a term, the documents that contain it. - * This is the inverse of the natural relationship, in which documents list terms. - * - *

    Types of Fields

    - * - *

    In Lucene, fields may be stored, in which case their text is stored in the index - * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field - * may be both stored and indexed. - * - *

    The text of a field may be tokenized into terms to be indexed, or the text of a field - * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is - * useful for certain identifier fields to be indexed literally. - * - *

    See the {@link org.apache.lucene.document.Field Field} java docs for more information on - * Fields. - * - *

    Segments

    - * - *

    Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a - * fully independent index, which could be searched separately. Indexes evolve by: - * - *

      - *
    1. Creating new segments for newly added documents. - *
    2. Merging existing segments. - *
    - * - *

    Searches may involve multiple segments and/or multiple indexes, each index potentially - * composed of a set of segments. - * - *

    Document Numbers

    - * - *

    Internally, Lucene refers to documents by an integer document number. The first - * document added to an index is numbered zero, and each subsequent document added gets a number one - * greater than the previous. - * - *

    Note that a document's number may change, so caution should be taken when storing these - * numbers outside of Lucene. In particular, numbers may change in the following situations: - * - *

      - *
    • - *

      The numbers stored in each segment are unique only within the segment, and must be - * converted before they can be used in a larger context. The standard technique is to - * allocate each segment a range of values, based on the range of numbers used in that - * segment. To convert a document number from a segment to an external value, the segment's - * base document number is added. To convert an external value back to a - * segment-specific value, the segment is identified by the range that the external value is - * in, and the segment's base value is subtracted. For example two five document segments - * might be combined, so that the first segment has a base value of zero, and the second of - * five. Document three from the second segment would have an external value of eight. - *

    • - *

      When documents are deleted, gaps are created in the numbering. These are eventually - * removed as the index evolves through merging. Deleted documents are dropped when segments - * are merged. A freshly-merged segment thus has no gaps in its numbering. - *

    - * - *
    - * - *

    Index Structure Overview

    - * - *
    - * - *

    Each segment index maintains the following: - * - *

      - *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This - * contains metadata about a segment, such as the number of documents, what files it uses, and - * information about how the segment is sorted - *
    • {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This - * contains metadata about the set of named fields used in the index. - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}. - * This contains, for each document, a list of attribute-value pairs, where the attributes are - * field names. These are used to store auxiliary information about the document, such as its - * title, url, or an identifier to access a database. The set of stored fields are what is - * returned for each hit when searching. This is keyed by document number. - *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term dictionary}. A - * dictionary containing all of the terms used in all of the indexed fields of all of the - * documents. The dictionary also contains the number of documents which contain the term, and - * pointers to the term's frequency and proximity data. - *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Frequency data}. For - * each term in the dictionary, the numbers of all the documents that contain that term, and - * the frequency of the term in that document, unless frequencies are omitted ({@link - * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) - *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Proximity data}. For - * each term in the dictionary, the positions that the term occurs in each document. Note that - * this will not exist if all fields in all documents omit position data. - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For - * each field in each document, a value is stored that is multiplied into the score for hits - * on that field. - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each - * field in each document, the term vector (sometimes called document vector) may be stored. A - * term vector consists of term text and term frequency. To add Term Vectors to your index see - * the {@link org.apache.lucene.document.Field Field} constructors - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like - * stored values, these are also keyed by document number, but are generally intended to be - * loaded into main memory for fast access. Whereas stored values are generally intended for - * summary results from searches, per-document values are useful for things like scoring - * factors. - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An - * optional file indicating which documents are live. - *
    • {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair - * of files, recording dimensionally indexed fields, to enable fast numeric range filtering - * and large numeric values like BigInteger and BigDecimal (1D) and geographic shape - * intersection (2D, 3D). - *
    • {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The - * vector format stores numeric vectors in a format optimized for random access and - * computation, supporting high-dimensional nearest-neighbor search. - *
    - * - *

    Details on each of these are provided in their linked pages.

    - * - *

    File Naming

    - * - *
    - * - *

    All files belonging to a segment have the same name with varying extensions. The extensions - * correspond to the different file formats described below. When using the Compound File format - * (default for small segments) these files (except for the Segment info file, the Lock file, and - * Deleted documents file) are collapsed into a single .cfs file (see below for details) - * - *

    Typically, all segments in an index are stored in a single directory, although this is not - * required. - * - *

    File names are never re-used. That is, when any file is saved to the Directory it is given a - * never before used filename. This is achieved using a simple generations approach. For example, - * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long - * integer represented in alpha-numeric (base 36) form.

    - * - *

    Summary of File Extensions

    - * - *
    - * - *

    The following table summarizes the names and extensions of the files in Lucene: - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    lucene filenames by extension
    NameExtensionBrief Description
    {@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
    Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
    {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}.siStores metadata about a segment
    {@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
    {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}.fnmStores information about the fields
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Index}.tipThe index into the Term Dictionary
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Positions}.posStores position information about where a term occurs in the index
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
    {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
    {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
    {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
    {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
    {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
    {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.dii, .dimHolds indexed points
    {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}.vec, .vem, .veq, vexHolds indexed vectors; .vec files contain the raw vector data, - * .vem the vector metadata, .veq the quantized vector data, and .vex the - * hnsw graph data.
    - * - *

    - * - *

    Lock File

    - * - * The write lock, which is stored in the index directory by default, is named "write.lock". If the - * lock directory is different from the index directory then the write lock will be named - * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index - * directory. When this file is present, a writer is currently modifying the index (adding or - * removing documents). This lock file ensures that only one writer is modifying the index at a - * time. - * - *

    History

    - * - *

    Compatibility notes are provided in this document, describing how file formats have changed - * from prior versions: - * - *

      - *
    • In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit - * lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching - * or adding/deleting of docs. When the new segments file is saved (committed), it will be - * written in the new file format (meaning no specific "upgrade" process is needed). But note - * that once a commit has occurred, pre-2.1 Lucene will not be able to read the index. - *
    • In version 2.3, the file format was changed to allow segments to share a single set of doc - * store (vectors & stored fields) files. This allows for faster indexing in certain - * cases. The change is fully backwards compatible (in the same way as the lock-less commits - * change in 2.1). - *
    • In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified - * UTF-8. See LUCENE-510 for - * details. - *
    • In version 2.9, an optional opaque Map<String,String> CommitUserData may be passed to - * IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N - * file. See LUCENE-1382 for - * details. Also, diagnostics were added to each segment written recording details about why - * it was written (due to flush, merge; which OS/JRE was used; etc.). See issue LUCENE-1654 for details. - *
    • In version 3.0, compressed fields are no longer written to the index (they can still be - * read, but on merge the new segment will write them, uncompressed). See issue LUCENE-1960 for details. - *
    • In version 3.1, segments records the code version that created them. See LUCENE-2720 for details. - * Additionally segments track explicitly whether or not they have term vectors. See LUCENE-2811 for details. - *
    • In version 3.2, numeric fields are written as natively to stored fields file, previously - * they were stored in text format only. - *
    • In version 3.4, fields can omit position data while still indexing term frequencies. - *
    • In version 4.0, the format of the inverted index became extensible via the {@link - * org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues}) - * was introduced. Normalization factors need no longer be a single byte, they can be any - * {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be - * unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into - * the postings lists. Payloads can be stored in the term vectors. - *
    • In version 4.1, the format of the postings list changed to use either of FOR compression or - * variable-byte encoding, depending upon the frequency of the term. Terms appearing only once - * were changed to inline directly into the term dictionary. Stored fields are compressed by - * default. - *
    • In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued - * type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields. - *
    • In version 4.5, DocValues were extended to explicitly represent missing values. - *
    • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to - * allow updating NumericDocValues fields. - *
    • In version 4.8, checksum footers were added to the end of each index file for improved data - * integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32 - * checksum of the file. - *
    • In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is - * suitable for faceting/sorting/analytics. - *
    • In version 5.4, DocValues have been improved to store more information on disk: addresses - * for binary fields and ord indexes for multi-valued fields. - *
    • In version 6.0, Points were added, for multi-dimensional range/distance search. - *
    • In version 6.2, new Segment info format that reads/writes the index sort, to support index - * sorting. - *
    • In version 7.0, DocValues have been improved to better support sparse doc values thanks to - * an iterator API. - *
    • In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term - * freq, normalization factor) pairs that may trigger the maximum score of the block. This - * information is recorded alongside skip data in order to be able to skip blocks of doc ids - * if they may not produce high enough scores. Additionally doc values and norms has been - * extended with jump-tables to make access O(1) instead of O(n), where n is the number of - * elements to skip when advancing in the data. - *
    • In version 8.4, postings, positions, offsets and payload lengths have move to a more - * performant encoding that is vectorized. - *
    • In version 8.6, index sort serialization is delegated to the sorts themselves, to allow - * user-defined sorts to be used - *
    • In version 8.7, stored fields compression became adaptive to better handle documents with - * smaller stored fields. - *
    • In version 9.0, vector-valued fields were added. - *
    • In version 9.1, vector-valued fields were modified to add a graph hierarchy. - *
    • In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by - * IndexDISI. ordToDoc mappings was added to .vem. - *
    • In version 9.5, HNSW graph connections were changed to be delta-encoded with vints. - * Additionally, metadata file size improvements were made by delta-encoding nodes by graph - * layer and not writing the node ids for the zeroth layer. - *
    • In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector - * format to utilize int8 quantized vectors for float32 vector search. - *
    - * - * - * - *

    Limitations

    - * - *
    - * - *

    Lucene uses a Java int to refer to document numbers, and the index file format - * uses an Int32 on-disk to store document numbers. This is a limitation of both the - * index file format and the current implementation. Eventually these should be replaced with either - * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt - * VInt} values which have no limit.

    - */ +/** Lucene 9.9 file format. */ package org.apache.lucene.codecs.lucene99; diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java index 55d0e228d1a..48ac987bee3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java @@ -21,8 +21,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -399,10 +399,10 @@ public class PhraseQuery extends Query { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. + * Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene99PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -411,7 +411,7 @@ public class PhraseQuery extends Query { /** * Number of simple operations in {@link - * Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill + * Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill * is done. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index 816097b3a66..8b672496601 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene99.Lucene99Codec +org.apache.lucene.codecs.lucene912.Lucene912Codec diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 0fd87daff7d..cd63926a287 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat +org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java index 8ca7fb8a41d..c72bcfeea46 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java @@ -18,7 +18,7 @@ package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; @@ -31,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase; public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec(Lucene99Codec.Mode.BEST_COMPRESSION); + return new Lucene912Codec(Lucene912Codec.Mode.BEST_COMPRESSION); } /** @@ -42,7 +42,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie for (int i = 0; i < 10; i++) { IndexWriterConfig iwc = newIndexWriterConfig(); iwc.setCodec( - new Lucene99Codec(RandomPicks.randomFrom(random(), Lucene99Codec.Mode.values()))); + new Lucene912Codec(RandomPicks.randomFrom(random(), Lucene912Codec.Mode.values()))); IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig()); Document doc = new Document(); doc.add(new StoredField("field1", "value1")); @@ -72,7 +72,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie expectThrows( NullPointerException.class, () -> { - new Lucene99Codec(null); + new Lucene912Codec(null); }); expectThrows( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java new file mode 100644 index 00000000000..1b8d0618c60 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java @@ -0,0 +1,157 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompetitiveImpactAccumulator; +import org.apache.lucene.codecs.lucene90.blocktree.FieldReader; +import org.apache.lucene.codecs.lucene90.blocktree.Stats; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader.MutableImpactList; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.analysis.MockAnalyzer; +import org.apache.lucene.tests.index.BasePostingsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestLucene912PostingsFormat extends BasePostingsFormatTestCase { + + @Override + protected Codec getCodec() { + return TestUtil.alwaysPostingsFormat(new Lucene912PostingsFormat()); + } + + public void testVInt15() throws IOException { + byte[] bytes = new byte[5]; + ByteArrayDataOutput out = new ByteArrayDataOutput(bytes); + ByteArrayDataInput in = new ByteArrayDataInput(); + for (int i : new int[] {0, 1, 127, 128, 32767, 32768, Integer.MAX_VALUE}) { + out.reset(bytes); + Lucene912PostingsWriter.writeVInt15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene912PostingsReader.readVInt15(in)); + assertEquals(out.getPosition(), in.getPosition()); + } + } + + public void testVLong15() throws IOException { + byte[] bytes = new byte[9]; + ByteArrayDataOutput out = new ByteArrayDataOutput(bytes); + ByteArrayDataInput in = new ByteArrayDataInput(); + for (long i : new long[] {0, 1, 127, 128, 32767, 32768, Integer.MAX_VALUE, Long.MAX_VALUE}) { + out.reset(bytes); + Lucene912PostingsWriter.writeVLong15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene912PostingsReader.readVLong15(in)); + assertEquals(out.getPosition(), in.getPosition()); + } + } + + /** Make sure the final sub-block(s) are not skipped. */ + public void testFinalBlock() throws Exception { + Directory d = newDirectory(); + IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random()))); + for (int i = 0; i < 25; i++) { + Document doc = new Document(); + doc.add(newStringField("field", Character.toString((char) (97 + i)), Field.Store.NO)); + doc.add(newStringField("field", "z" + Character.toString((char) (97 + i)), Field.Store.NO)); + w.addDocument(doc); + } + w.forceMerge(1); + + DirectoryReader r = DirectoryReader.open(w); + assertEquals(1, r.leaves().size()); + FieldReader field = (FieldReader) r.leaves().get(0).reader().terms("field"); + // We should see exactly two blocks: one root block (prefix empty string) and one block for z* + // terms (prefix z): + Stats stats = field.getStats(); + assertEquals(0, stats.floorBlockCount); + assertEquals(2, stats.nonFloorBlockCount); + r.close(); + w.close(); + d.close(); + } + + public void testImpactSerialization() throws IOException { + // omit norms and omit freqs + doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); + + // omit freqs + doTestImpactSerialization(Collections.singletonList(new Impact(1, 42L))); + // omit freqs with very large norms + doTestImpactSerialization(Collections.singletonList(new Impact(1, -100L))); + + // omit norms + doTestImpactSerialization(Collections.singletonList(new Impact(30, 1L))); + // omit norms with large freq + doTestImpactSerialization(Collections.singletonList(new Impact(500, 1L))); + + // freqs and norms, basic + doTestImpactSerialization( + Arrays.asList( + new Impact(1, 7L), + new Impact(3, 9L), + new Impact(7, 10L), + new Impact(15, 11L), + new Impact(20, 13L), + new Impact(28, 14L))); + + // freqs and norms, high values + doTestImpactSerialization( + Arrays.asList( + new Impact(2, 2L), + new Impact(10, 10L), + new Impact(12, 50L), + new Impact(50, -100L), + new Impact(1000, -80L), + new Impact(1005, -3L))); + } + + private void doTestImpactSerialization(List impacts) throws IOException { + CompetitiveImpactAccumulator acc = new CompetitiveImpactAccumulator(); + for (Impact impact : impacts) { + acc.add(impact.freq, impact.norm); + } + try (Directory dir = newDirectory()) { + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + Lucene912PostingsWriter.writeImpacts(acc.getCompetitiveFreqNormPairs(), out); + } + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + byte[] b = new byte[Math.toIntExact(in.length())]; + in.readBytes(b, 0, b.length); + List impacts2 = + Lucene912PostingsReader.readImpacts( + new ByteArrayDataInput(b), + new MutableImpactList(impacts.size() + random().nextInt(3))); + assertEquals(impacts, impacts2); + } + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java index e26284eb70b..8e69e833b98 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java @@ -29,6 +29,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -79,7 +80,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; @@ -95,7 +96,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene99Codec() { + new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java index 4d5073ef1e9..58e6c27e326 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java @@ -27,6 +27,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -51,7 +52,7 @@ import org.apache.lucene.util.quantization.ScalarQuantizer; public class TestLucene99ScalarQuantizedVectorScorer extends LuceneTestCase { private static Codec getCodec(int bits, boolean compress) { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java index 107c39eed07..b221cb19dde 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java @@ -28,6 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -67,7 +68,7 @@ public class TestLucene99ScalarQuantizedVectorsFormat extends BaseKnnVectorsForm @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java deleted file mode 100644 index 310d8524be9..00000000000 --- a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * 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.index; - -import java.io.IOException; -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.PhraseQuery; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.store.ByteBuffersDirectory; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FilterIndexInput; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.tests.analysis.MockAnalyzer; -import org.apache.lucene.tests.analysis.MockTokenizer; -import org.apache.lucene.tests.store.MockDirectoryWrapper; -import org.apache.lucene.tests.util.LuceneTestCase; -import org.apache.lucene.tests.util.TestUtil; -import org.apache.lucene.util.BytesRef; - -/** Tests lazy skipping on the proximity file. */ -public class TestLazyProxSkipping extends LuceneTestCase { - private IndexSearcher searcher; - private int seeksCounter = 0; - - private String field = "tokens"; - private String term1 = "xx"; - private String term2 = "yy"; - private String term3 = "zz"; - - private class SeekCountingDirectory extends MockDirectoryWrapper { - public SeekCountingDirectory(Directory delegate) { - super(random(), delegate); - } - - @Override - public IndexInput openInput(String name, IOContext context) throws IOException { - IndexInput ii = super.openInput(name, context); - if (name.endsWith(".prx") || name.endsWith(".pos")) { - // we decorate the proxStream with a wrapper class that allows to count the number of calls - // of seek() - ii = new SeeksCountingStream(ii); - } - return ii; - } - } - - private void createIndex(int numHits) throws IOException { - int numDocs = 500; - - final Analyzer analyzer = - new Analyzer() { - @Override - public TokenStreamComponents createComponents(String fieldName) { - return new TokenStreamComponents(new MockTokenizer(MockTokenizer.WHITESPACE, true)); - } - }; - Directory directory = new SeekCountingDirectory(new ByteBuffersDirectory()); - // note: test explicitly disables payloads - IndexWriter writer = - new IndexWriter( - directory, - newIndexWriterConfig(analyzer) - .setMaxBufferedDocs(10) - .setMergePolicy(newLogMergePolicy(false))); - - for (int i = 0; i < numDocs; i++) { - Document doc = new Document(); - String content; - if (i % (numDocs / numHits) == 0) { - // add a document that matches the query "term1 term2" - content = this.term1 + " " + this.term2; - } else if (i % 15 == 0) { - // add a document that only contains term1 - content = this.term1 + " " + this.term1; - } else { - // add a document that contains term2 but not term 1 - content = this.term3 + " " + this.term2; - } - - doc.add(newTextField(this.field, content, Field.Store.YES)); - writer.addDocument(doc); - } - - // make sure the index has only a single segment - writer.forceMerge(1); - writer.close(); - - LeafReader reader = getOnlyLeafReader(DirectoryReader.open(directory)); - - this.searcher = newSearcher(reader); - } - - private ScoreDoc[] search() throws IOException { - // create PhraseQuery "term1 term2" and search - PhraseQuery pq = new PhraseQuery(field, term1, term2); - return this.searcher.search(pq, 1000).scoreDocs; - } - - private void performTest(int numHits) throws IOException { - createIndex(numHits); - this.seeksCounter = 0; - ScoreDoc[] hits = search(); - // verify that the right number of docs was found - assertEquals(numHits, hits.length); - - // check if the number of calls of seek() does not exceed the number of hits - assertTrue(this.seeksCounter > 0); - assertTrue( - "seeksCounter=" + this.seeksCounter + " numHits=" + numHits, - this.seeksCounter <= numHits + 1); - searcher.getIndexReader().close(); - } - - public void testLazySkipping() throws IOException { - final String fieldFormat = TestUtil.getPostingsFormat(this.field); - assumeFalse("This test cannot run with Direct postings format", fieldFormat.equals("Direct")); - assumeFalse( - "This test cannot run with SimpleText postings format", fieldFormat.equals("SimpleText")); - - // test whether only the minimum amount of seeks() - // are performed - performTest(5); - performTest(10); - } - - public void testSeek() throws IOException { - Directory directory = newDirectory(); - IndexWriter writer = - new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random()))); - for (int i = 0; i < 10; i++) { - Document doc = new Document(); - doc.add(newTextField(this.field, "a b", Field.Store.YES)); - writer.addDocument(doc); - } - - writer.close(); - IndexReader reader = DirectoryReader.open(directory); - - PostingsEnum tp = MultiTerms.getTermPostingsEnum(reader, this.field, new BytesRef("b")); - - for (int i = 0; i < 10; i++) { - tp.nextDoc(); - assertEquals(tp.docID(), i); - assertEquals(tp.nextPosition(), 1); - } - - tp = MultiTerms.getTermPostingsEnum(reader, this.field, new BytesRef("a")); - - for (int i = 0; i < 10; i++) { - tp.nextDoc(); - assertEquals(tp.docID(), i); - assertEquals(tp.nextPosition(), 0); - } - reader.close(); - directory.close(); - } - - // Simply extends IndexInput in a way that we are able to count the number - // of invocations of seek() - class SeeksCountingStream extends FilterIndexInput { - - SeeksCountingStream(IndexInput input) { - super("SeekCountingStream(" + input + ")", input); - } - - @Override - public void seek(long pos) throws IOException { - TestLazyProxSkipping.this.seeksCounter++; - in.seek(pos); - } - - @Override - public SeeksCountingStream clone() { - return new SeeksCountingStream(in.clone()); - } - } -} diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java index 2a5d8fffd01..e5ca29b9fae 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Objects; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.PostingsEnum; @@ -261,10 +261,10 @@ class TermIntervalsSource extends IntervalsSource { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene99PostingsReader.EverythingEnum#nextPosition()}. + * Lucene912PostingsReader.EverythingEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene99PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -272,7 +272,7 @@ class TermIntervalsSource extends IntervalsSource { private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128; /** - * Number of simple operations in {@link Lucene99PostingsReader.EverythingEnum#nextPosition()} + * Number of simple operations in {@link Lucene912PostingsReader.EverythingEnum#nextPosition()} * when no seek or buffer refill is done. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/suggest/src/java/module-info.java b/lucene/suggest/src/java/module-info.java index 673fa391213..c3f5b7c178f 100644 --- a/lucene/suggest/src/java/module-info.java +++ b/lucene/suggest/src/java/module-info.java @@ -31,7 +31,8 @@ module org.apache.lucene.suggest { org.apache.lucene.search.suggest.document.Completion50PostingsFormat, org.apache.lucene.search.suggest.document.Completion84PostingsFormat, org.apache.lucene.search.suggest.document.Completion90PostingsFormat, - org.apache.lucene.search.suggest.document.Completion99PostingsFormat; + org.apache.lucene.search.suggest.document.Completion99PostingsFormat, + org.apache.lucene.search.suggest.document.Completion912PostingsFormat; provides org.apache.lucene.analysis.TokenFilterFactory with org.apache.lucene.search.suggest.analyzing.SuggestStopFilterFactory; } diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java new file mode 100644 index 00000000000..341b034e8e8 --- /dev/null +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java @@ -0,0 +1,45 @@ +/* + * 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.search.suggest.document; + +import org.apache.lucene.codecs.PostingsFormat; + +/** + * {@link CompletionPostingsFormat} for {@link + * org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat} + * + * @lucene.experimental + */ +public class Completion912PostingsFormat extends CompletionPostingsFormat { + /** Creates a {@link Completion912PostingsFormat} that will load the completion FST on-heap. */ + public Completion912PostingsFormat() { + this(FSTLoadMode.ON_HEAP); + } + + /** + * Creates a {@link Completion912PostingsFormat} that will use the provided fstLoadMode + * to determine if the completion FST should be loaded on or off heap. + */ + public Completion912PostingsFormat(FSTLoadMode fstLoadMode) { + super("Completion912", fstLoadMode); + } + + @Override + protected PostingsFormat delegatePostingsFormat() { + return PostingsFormat.forName("Lucene912"); + } +} diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java index 4449d754e29..d3ea76d5425 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java @@ -19,8 +19,10 @@ package org.apache.lucene.search.suggest.document; import org.apache.lucene.codecs.PostingsFormat; /** - * {@link CompletionPostingsFormat} for {@link - * org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat} + * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code + * org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat}. This format is only used for + * backward-compatibility of the index format and cannot be used to write data, use {@link + * Completion912PostingsFormat} on new indices. * * @lucene.experimental */ diff --git a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 81d1ed9465d..8544a97b88f 100644 --- a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -34,3 +34,4 @@ org.apache.lucene.search.suggest.document.Completion50PostingsFormat org.apache.lucene.search.suggest.document.Completion84PostingsFormat org.apache.lucene.search.suggest.document.Completion90PostingsFormat org.apache.lucene.search.suggest.document.Completion99PostingsFormat +org.apache.lucene.search.suggest.document.Completion912PostingsFormat diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java index ca5eff9cc5a..2d6e9471c98 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java @@ -951,7 +951,7 @@ public class TestSuggestField extends LuceneTestCase { new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { final CompletionPostingsFormat.FSTLoadMode fstLoadMode = RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); - final PostingsFormat postingsFormat = new Completion99PostingsFormat(fstLoadMode); + final PostingsFormat postingsFormat = new Completion912PostingsFormat(fstLoadMode); @Override public PostingsFormat postingsFormat() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java index 0a0c8f673db..9688c6c0e35 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java @@ -28,9 +28,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter; import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -38,7 +38,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. + * Customized version of {@link Lucene912PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. */ public final class LuceneFixedGap extends PostingsFormat { final int termIndexInterval; @@ -54,7 +54,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -91,7 +91,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java index a864263b7c3..1fafc8f448c 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link + * Customized version of {@link Lucene912PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval, but forcing high docfreq terms to be indexed * terms. */ @@ -59,7 +59,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -100,7 +100,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java index 667acf0ceb3..bfaf0914651 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link + * Customized version of {@link Lucene912PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval. */ public final class LuceneVarGapFixedInterval extends PostingsFormat { @@ -56,7 +56,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -95,7 +95,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java index 10e5ec96332..6b10e7b19f4 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java @@ -37,8 +37,8 @@ import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.codecs.memory.FSTTermsReader; import org.apache.lucene.codecs.memory.FSTTermsWriter; import org.apache.lucene.index.FieldInfo; @@ -121,7 +121,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { random.nextInt(); // consume a random for buffersize - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); final FieldsConsumer fields; final int t1 = random.nextInt(4); @@ -289,7 +289,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); } - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); final FieldsProducer fields; final int t1 = random.nextInt(4); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java index 0c469b392a2..4b13531327b 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockEncoder; import org.apache.lucene.codecs.uniformsplit.IndexDictionary; @@ -67,7 +67,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(segmentWriteState); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(segmentWriteState); boolean success = false; try { FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter); @@ -145,7 +145,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(segmentReadState); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(segmentReadState); boolean success = false; try { FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java index c386a7ef85e..378444e394a 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java @@ -38,7 +38,7 @@ import java.util.TimeZone; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.tests.codecs.asserting.AssertingCodec; @@ -188,9 +188,9 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule { } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) { codec = CompressingCodec.randomInstance(random); - } else if ("Lucene99".equals(TEST_CODEC) - || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene99"))) { - codec = new Lucene99Codec(RandomPicks.randomFrom(random, Lucene99Codec.Mode.values())); + } else if ("Lucene912".equals(TEST_CODEC) + || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene912"))) { + codec = new Lucene912Codec(RandomPicks.randomFrom(random, Lucene912Codec.Mode.values())); } else if (!"random".equals(TEST_CODEC)) { codec = Codec.forName(TEST_CODEC); } else if ("random".equals(TEST_POSTINGSFORMAT)) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java index 8bed935f7db..195b5563958 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java @@ -56,9 +56,9 @@ import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.document.BinaryDocValuesField; @@ -1231,7 +1231,7 @@ public final class TestUtil { * different from {@link Codec#getDefault()} because that is randomized. */ public static Codec getDefaultCodec() { - return new Lucene99Codec(); + return new Lucene912Codec(); } /** @@ -1239,7 +1239,7 @@ public final class TestUtil { * Lucene. */ public static PostingsFormat getDefaultPostingsFormat() { - return new Lucene99PostingsFormat(); + return new Lucene912PostingsFormat(); } /** @@ -1250,7 +1250,7 @@ public final class TestUtil { */ public static PostingsFormat getDefaultPostingsFormat( int minItemsPerBlock, int maxItemsPerBlock) { - return new Lucene99PostingsFormat(minItemsPerBlock, maxItemsPerBlock); + return new Lucene912PostingsFormat(minItemsPerBlock, maxItemsPerBlock); } /** Returns a random postings format that supports term ordinals */ From a1816e3f65cda4ba4ba6c518efcf75e5b96fbeff Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 31 Jul 2024 19:34:35 +0200 Subject: [PATCH 32/66] Wrap Executor in TaskExecutor to never reject (#13622) Make it so rejected tasks are execute right away on the caller thread. Users of the API shouldn't have to worry about rejections when we don't expose any upper limit to the task count that we put on the executor that would help in sizing a queue for the executor. --- .../apache/lucene/search/TaskExecutor.java | 16 ++++++++++++++- .../lucene/search/TestTaskExecutor.java | 20 +++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java index 8832cf478e9..9d2176fd3af 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java +++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java @@ -28,6 +28,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Future; import java.util.concurrent.FutureTask; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RunnableFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -53,7 +54,20 @@ public final class TaskExecutor { * @param executor the executor to be used for running tasks concurrently */ public TaskExecutor(Executor executor) { - this.executor = Objects.requireNonNull(executor, "Executor is null"); + Objects.requireNonNull(executor, "Executor is null"); + this.executor = + r -> { + try { + executor.execute(r); + } catch ( + @SuppressWarnings("unused") + RejectedExecutionException rejectedExecutionException) { + // execute directly on the current thread in case of rejection to ensure a rejecting + // executor only reduces parallelism and does not + // result in failure + r.run(); + } + }; } /** diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java index 1949afbd51c..26847f60a2a 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java @@ -363,4 +363,24 @@ public class TestTaskExecutor extends LuceneTestCase { assertEquals(0, throwable.getSuppressed().length); assertEquals(throwingTask, executedTasks.get()); } + + public void testTaskRejectionDoesNotFailExecution() throws Exception { + try (ThreadPoolExecutor threadPoolExecutor = + new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, new ArrayBlockingQueue<>(1))) { + final int taskCount = 1000; // enough tasks to cause queuing and rejections on the executor + final ArrayList> callables = new ArrayList<>(taskCount); + final AtomicInteger executedTasks = new AtomicInteger(0); + for (int i = 0; i < taskCount; i++) { + callables.add( + () -> { + executedTasks.incrementAndGet(); + return null; + }); + } + final TaskExecutor taskExecutor = new TaskExecutor(threadPoolExecutor); + var res = taskExecutor.invokeAll(callables); + assertEquals(taskCount, res.size()); + assertEquals(taskCount, executedTasks.get()); + } + } } From 255a2fcf9ced25e4d9615cc1304f8224522c2e5a Mon Sep 17 00:00:00 2001 From: Jakub Slowinski <32519034+slow-J@users.noreply.github.com> Date: Wed, 31 Jul 2024 19:00:34 +0100 Subject: [PATCH 33/66] Fix comments containing 'the this' (#13624) --- lucene/facet/src/java/org/apache/lucene/facet/FacetField.java | 2 +- .../join/src/java/org/apache/lucene/search/join/JoinUtil.java | 2 +- .../src/java/org/apache/lucene/index/memory/MemoryIndex.java | 2 +- .../org/apache/lucene/search/suggest/document/SuggestField.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetField.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetField.java index a0fc42599c6..225d8e14e28 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/FacetField.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetField.java @@ -42,7 +42,7 @@ public class FacetField extends Field { /** Path for this field. */ public final String[] path; - /** Creates the this from {@code dim} and {@code path}. */ + /** Creates this from {@code dim} and {@code path}. */ public FacetField(String dim, String... path) { super("dummy", TYPE); verifyLabel(dim); diff --git a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java index 079baa13942..2cf4c394460 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java @@ -470,7 +470,7 @@ public final class JoinUtil { *

    This join has certain restrictions and requirements: 1) A document can only refer to one * other document. (but can be referred by one or more documents) 2) Documents on each side of the * join must be distinguishable. Typically this can be done by adding an extra field that - * identifies the "from" and "to" side and then the fromQuery and toQuery must take the this into + * identifies the "from" and "to" side and then the fromQuery and toQuery must take this into * account. 3) There must be a single sorted doc values join field used by both the "from" and * "to" documents. This join field should store the join values as UTF-8 strings. 4) An ordinal * map must be provided that is created on top of the join field. diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index ff65bca6a16..08fb1cf6b5b 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -281,7 +281,7 @@ public class MemoryIndex { /** * Returns the offset of the currently written slice. The returned value should be used as the * end offset to initialize a {@link SliceReader} once this slice is fully written or to reset - * the this writer if another slice needs to be written. + * the writer if another slice needs to be written. */ public int getCurrentOffset() { return offset; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java index 82484e8a595..a595197ca79 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java @@ -42,7 +42,7 @@ import org.apache.lucene.util.BytesRef; * document.add(new SuggestField(name, "suggestion", 4)); * * - * To perform document suggestions based on the this field, use {@link + * To perform document suggestions based on the field, use {@link * SuggestIndexSearcher#suggest(CompletionQuery, int, boolean)} * * @lucene.experimental From ca098e63b972285b250e7d9845b7e495efb4fdca Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 31 Jul 2024 20:31:13 +0200 Subject: [PATCH 34/66] Deduplicate bytes for `FieldReader#rootCode` (#13610) Looking at how these instances are serialized to disk it appears that the empty output in the FST metadata is always the same as the rootCode bytes. Without changing the serialization we could at least deduplicate here, saving hundreds of MB in some high-segment count use cases I observed in ES. --- .../lucene/codecs/lucene90/blocktree/FieldReader.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java index ed3e827c37c..259639762ec 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java @@ -78,7 +78,6 @@ public final class FieldReader extends Terms { this.sumTotalTermFreq = sumTotalTermFreq; this.sumDocFreq = sumDocFreq; this.docCount = docCount; - this.rootCode = rootCode; this.minTerm = minTerm; this.maxTerm = maxTerm; // if (DEBUG) { @@ -100,6 +99,14 @@ public final class FieldReader extends Terms { w.close(); } */ + BytesRef emptyOutput = metadata.getEmptyOutput(); + if (rootCode.equals(emptyOutput) == false) { + // TODO: this branch is never taken + assert false; + this.rootCode = rootCode; + } else { + this.rootCode = emptyOutput; + } } long readVLongOutput(DataInput in) throws IOException { From 47650a4314d5ec16a20ab7f8d91df5b52ad9dc3e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 31 Jul 2024 20:38:21 +0200 Subject: [PATCH 35/66] Deduplicate min and max term in single-term FieldReader (#13618) I noticed that single-term readers are an edge case but not that uncommon in Elasticsearch heap dumps. It seems quite common to have a constant value for some field across a complete segment (e.g. a version value that is repeated endlessly in logs). Seems simple enough to deduplicate here to save a couple MB of heap. --- .../lucene90/blocktree/Lucene90BlockTreeTermsReader.java | 5 +++++ .../lucene90/blocktree/Lucene90BlockTreeTermsWriter.java | 2 -- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java index c2ff50215a3..8be0b1e0f4a 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java @@ -200,6 +200,11 @@ public final class Lucene90BlockTreeTermsReader extends FieldsProducer { final int docCount = metaIn.readVInt(); BytesRef minTerm = readBytesRef(metaIn); BytesRef maxTerm = readBytesRef(metaIn); + if (numTerms == 1) { + assert maxTerm.equals(minTerm); + // save heap for edge case of a single term only so min == max + maxTerm = minTerm; + } if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs throw new CorruptIndexException( diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java index 90b34750463..8d0d5aaaa22 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java @@ -598,8 +598,6 @@ public final class Lucene90BlockTreeTermsWriter extends FieldsConsumer { private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance(); private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder(); - static final BytesRef EMPTY_BYTES_REF = new BytesRef(); - private static class StatsWriter { private final DataOutput out; From 0a2476985044a7a7de9bb955546d4c8130893776 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 1 Aug 2024 12:46:06 +0200 Subject: [PATCH 36/66] Don't clone an IndexInput if postings are inlined in the terms dict (#13585). --- .../lucene912/Lucene912PostingsReader.java | 87 +++++++++++++------ 1 file changed, 59 insertions(+), 28 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index f0f22f42663..6d1336d3750 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -366,7 +366,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private int level0LastDocID; // level 1 skip data private int level1LastDocID; - private long level1DocEndOffset; + private long level1DocEndFP; private int level1DocCountUpto; private boolean needsFreq; // true if the caller actually needs frequencies @@ -409,11 +409,13 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docFreq = termState.docFreq; totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; singletonDocID = termState.singletonDocID; - if (docIn == null) { - // lazy init - docIn = startDocIn.clone(); + if (docFreq > 1) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); } - prefetchPostings(docIn, termState); doc = -1; this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS); @@ -425,8 +427,15 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { prevDocID = -1; docCountUpto = 0; level0LastDocID = -1; - level1LastDocID = -1; - level1DocEndOffset = termState.docStartFP; + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } level1DocCountUpto = 0; docBufferUpto = BLOCK_SIZE; freqFP = -1; @@ -510,7 +519,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { while (true) { prevDocID = level1LastDocID; level0LastDocID = level1LastDocID; - docIn.seek(level1DocEndOffset); + docIn.seek(level1DocEndFP); docCountUpto = level1DocCountUpto; level1DocCountUpto += LEVEL1_NUM_DOCS; @@ -520,7 +529,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } level1LastDocID += docIn.readVInt(); - level1DocEndOffset = docIn.readVLong() + docIn.getFilePointer(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); if (level1LastDocID >= target) { if (indexHasFreq) { @@ -677,7 +686,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private int level0LastDocID; private long level0PosEndFP; private int level0BlockPosUpto; - private long levelPayEndFP; + private long level0PayEndFP; private int level0BlockPayUpto; // level 1 skip data private int level1LastDocID; @@ -754,7 +763,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { payTermStartFP = termState.payStartFP; totalTermFreq = termState.totalTermFreq; singletonDocID = termState.singletonDocID; - if (docFreq > 1 || true) { + if (docFreq > 1) { if (docIn == null) { // lazy init docIn = startDocIn.clone(); @@ -768,7 +777,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { level1PosEndFP = posTermStartFP; level1PayEndFP = payTermStartFP; level0PosEndFP = posTermStartFP; - levelPayEndFP = payTermStartFP; + level0PayEndFP = payTermStartFP; posPendingCount = 0; payloadByteUpto = 0; if (termState.totalTermFreq < BLOCK_SIZE) { @@ -786,8 +795,15 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { prevDocID = -1; docCountUpto = 0; level0LastDocID = -1; - level1LastDocID = -1; - level1DocEndFP = termState.docStartFP; + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } level1DocCountUpto = 0; level1BlockPosUpto = 0; level1BlockPayUpto = 0; @@ -841,7 +857,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { level0PosEndFP = level1PosEndFP; level0BlockPosUpto = level1BlockPosUpto; if (indexHasOffsetsOrPayloads) { - levelPayEndFP = level1PayEndFP; + level0PayEndFP = level1PayEndFP; level0BlockPayUpto = level1BlockPayUpto; } docCountUpto = level1DocCountUpto; @@ -885,8 +901,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { posIn.seek(level0PosEndFP); posPendingCount = level0BlockPosUpto; if (indexHasOffsetsOrPayloads) { - assert levelPayEndFP >= payIn.getFilePointer(); - payIn.seek(levelPayEndFP); + assert level0PayEndFP >= payIn.getFilePointer(); + payIn.seek(level0PayEndFP); payloadByteUpto = level0BlockPayUpto; } posBufferUpto = BLOCK_SIZE; @@ -902,7 +918,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { level0PosEndFP += docIn.readVLong(); level0BlockPosUpto = docIn.readByte(); if (indexHasOffsetsOrPayloads) { - levelPayEndFP += docIn.readVLong(); + level0PayEndFP += docIn.readVLong(); level0BlockPayUpto = docIn.readVInt(); } } else { @@ -939,8 +955,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { posIn.seek(level0PosEndFP); posPendingCount = level0BlockPosUpto; if (indexHasOffsetsOrPayloads) { - assert levelPayEndFP >= payIn.getFilePointer(); - payIn.seek(levelPayEndFP); + assert level0PayEndFP >= payIn.getFilePointer(); + payIn.seek(level0PayEndFP); payloadByteUpto = level0BlockPayUpto; } posBufferUpto = BLOCK_SIZE; @@ -962,7 +978,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { level0PosEndFP += docIn.readVLong(); level0BlockPosUpto = docIn.readByte(); if (indexHasOffsetsOrPayloads) { - levelPayEndFP += docIn.readVLong(); + level0PayEndFP += docIn.readVLong(); level0BlockPayUpto = docIn.readVInt(); } @@ -1246,7 +1262,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; docFreq = termState.docFreq; - if (docFreq > 1 || true) { + if (docFreq > 1) { if (docIn == null) { // lazy init docIn = startDocIn.clone(); @@ -1263,8 +1279,15 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { prevDocID = -1; docCountUpto = 0; level0LastDocID = -1; - level1LastDocID = -1; - level1DocEndFP = termState.docStartFP; + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } level1DocCountUpto = 0; docBufferUpto = BLOCK_SIZE; freqFP = -1; @@ -1626,7 +1649,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { posTermStartFP = termState.posStartFP; totalTermFreq = termState.totalTermFreq; singletonDocID = termState.singletonDocID; - if (docFreq > 1 || true) { + if (docFreq > 1) { if (docIn == null) { // lazy init docIn = startDocIn.clone(); @@ -1649,8 +1672,15 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { prevDocID = -1; docCountUpto = 0; level0LastDocID = -1; - level1LastDocID = -1; - level1DocEndFP = termState.docStartFP; + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } level1DocCountUpto = 0; level1BlockPosUpto = 0; docBufferUpto = BLOCK_SIZE; @@ -2020,7 +2050,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } private void prefetchPostings(IndexInput docIn, IntBlockTermState state) throws IOException { - if (state.docFreq > 1 && docIn.getFilePointer() != state.docStartFP) { + assert state.docFreq > 1; // Singletons are inlined in the terms dict, nothing to prefetch + if (docIn.getFilePointer() != state.docStartFP) { // Don't prefetch if the input is already positioned at the right offset, which suggests that // the caller is streaming the entire inverted index (e.g. for merging), let the read-ahead // logic do its work instead. Note that this heuristic doesn't work for terms that have skip From e8eba4d455a644b883c5568a7af126651c8ad30f Mon Sep 17 00:00:00 2001 From: Egor Potemkin Date: Thu, 1 Aug 2024 15:11:58 +0100 Subject: [PATCH 37/66] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. (#13559) --- lucene/CHANGES.txt | 3 ++ .../java/org/apache/lucene/util/BitSet.java | 6 +++ .../org/apache/lucene/util/FixedBitSet.java | 26 ++++++++++++ .../apache/lucene/util/SparseFixedBitSet.java | 41 +++++++++++++++++-- .../lucene/search/join/BlockJoinSelector.java | 14 +++++-- .../lucene/tests/util/BaseBitSetTestCase.java | 24 +++++++++++ 6 files changed, 107 insertions(+), 7 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d23c491fe32..d65b9f847eb 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -108,8 +108,11 @@ API Changes * GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand) +* GITHUB#13559: Add BitSet#nextSetBit(int, int) to get the index of the first set bit in range. (Egor Potemkin) + * GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) + New Features --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/util/BitSet.java b/lucene/core/src/java/org/apache/lucene/util/BitSet.java index c5d84833b28..4946957caf1 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/BitSet.java @@ -92,6 +92,12 @@ public abstract class BitSet implements Bits, Accountable { */ public abstract int nextSetBit(int index); + /** + * Returns the index of the first set bit from start (inclusive) until end (exclusive). {@link + * DocIdSetIterator#NO_MORE_DOCS} is returned if there are no more set bits. + */ + public abstract int nextSetBit(int start, int end); + /** Assert that the current doc is -1. */ protected final void checkUnpositioned(DocIdSetIterator iter) { if (iter.docID() != -1) { diff --git a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java index 3dd1af5cbfe..750c6684e95 100644 --- a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java @@ -291,6 +291,32 @@ public final class FixedBitSet extends BitSet { return DocIdSetIterator.NO_MORE_DOCS; } + @Override + public int nextSetBit(int start, int upperBound) { + // Depends on the ghost bits being clear! + assert start >= 0 && start < numBits : "index=" + start + ", numBits=" + numBits; + assert start < upperBound : "index=" + start + ", upperBound=" + upperBound; + assert upperBound <= numBits : "upperBound=" + upperBound + ", numBits=" + numBits; + int i = start >> 6; + long word = bits[i] >> start; // skip all the bits to the right of index + + if (word != 0) { + int res = start + Long.numberOfTrailingZeros(word); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + + int limit = bits2words(upperBound); + while (++i < limit) { + word = bits[i]; + if (word != 0) { + int res = (i << 6) + Long.numberOfTrailingZeros(word); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + } + + return DocIdSetIterator.NO_MORE_DOCS; + } + @Override public int prevSetBit(int index) { assert index >= 0 && index < numBits : "index=" + index + " numBits=" + numBits; diff --git a/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java b/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java index b4ebe3cfc59..66805d53d94 100644 --- a/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java @@ -320,9 +320,11 @@ public class SparseFixedBitSet extends BitSet { } /** Return the first document that occurs on or after the provided block index. */ - private int firstDoc(int i4096) { + private int firstDoc(int i4096, int i4096upper) { + assert i4096upper <= indices.length + : "i4096upper=" + i4096 + ", indices.length=" + indices.length; long index = 0; - while (i4096 < indices.length) { + while (i4096 < i4096upper) { index = indices[i4096]; if (index != 0) { final int i64 = Long.numberOfTrailingZeros(index); @@ -353,7 +355,7 @@ public class SparseFixedBitSet extends BitSet { final long indexBits = index >>> i64 >>> 1; if (indexBits == 0) { // no more bits are set in the current block of 4096 bits, go to the next one - return firstDoc(i4096 + 1); + return firstDoc(i4096 + 1, indices.length); } // there are still set bits i64 += 1 + Long.numberOfTrailingZeros(indexBits); @@ -361,6 +363,39 @@ public class SparseFixedBitSet extends BitSet { return (i64 << 6) | Long.numberOfTrailingZeros(bits); } + @Override + public int nextSetBit(int start, int upperBound) { + assert start < length; + assert upperBound > start; + final int i4096 = start >>> 12; + final long index = indices[i4096]; + final long[] bitArray = this.bits[i4096]; + int i64 = start >>> 6; + final long i64bit = 1L << i64; + int o = Long.bitCount(index & (i64bit - 1)); + if ((index & i64bit) != 0) { + // There is at least one bit that is set in the current long, check if + // one of them is after i + final long bits = bitArray[o] >>> start; // shifts are mod 64 + if (bits != 0) { + int res = start + Long.numberOfTrailingZeros(bits); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + o += 1; + } + final long indexBits = index >>> i64 >>> 1; + if (indexBits == 0) { + // no more bits are set in the current block of 4096 bits, go to the next one + int res = firstDoc(i4096 + 1, blockCount(upperBound)); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + // there are still set bits + i64 += 1 + Long.numberOfTrailingZeros(indexBits); + final long bits = bitArray[o]; + int res = (i64 << 6) | Long.numberOfTrailingZeros(bits); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + /** Return the last document that occurs on or before the provided block index. */ private int lastDoc(int i4096) { long index; diff --git a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java index 1afca98288c..1da660ffa3b 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java @@ -64,14 +64,20 @@ public class BlockJoinSelector { return false; } - final int firstChild = parents.prevSetBit(docID - 1) + 1; - for (int child = children.nextSetBit(firstChild); - child < docID; - child = children.nextSetBit(child + 1)) { + final int firstPotentialChild = parents.prevSetBit(docID - 1) + 1; + assert firstPotentialChild <= docID; + if (firstPotentialChild == docID) { + // no children + return false; + } + for (int child = children.nextSetBit(firstPotentialChild, docID); + child != DocIdSetIterator.NO_MORE_DOCS; + child = children.nextSetBit(child + 1, docID)) { if (docsWithValue.get(child)) { return true; } } + return false; } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java index cbce97d87ac..095bcabbe4c 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java @@ -16,6 +16,7 @@ */ package org.apache.lucene.tests.util; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; import java.util.Collection; import java.util.Collections; @@ -104,6 +105,23 @@ public abstract class BaseBitSetTestCase extends LuceneTestCas } } + /** Test {@link BitSet#nextSetBit(int, int)}. */ + public void testNextSetBitInRange() throws IOException { + Random random = random(); + final int numBits = 1 + random().nextInt(100000); + for (float percentSet : new float[] {0, 0.01f, 0.1f, 0.5f, 0.9f, 0.99f, 1f}) { + BitSet set1 = new JavaUtilBitSet(randomSet(numBits, percentSet), numBits); + T set2 = copyOf(set1, numBits); + for (int start = 0; start < numBits; ++start) { + int end = RandomNumbers.randomIntBetween(random, start + 1, numBits); + assertEquals( + "start=" + start + ", end=" + end + ", numBits=" + numBits, + set1.nextSetBit(start, end), + set2.nextSetBit(start, end)); + } + } + } + /** Test the {@link BitSet#set} method. */ public void testSet() throws IOException { Random random = random(); @@ -330,5 +348,11 @@ public abstract class BaseBitSetTestCase extends LuceneTestCas } return next; } + + @Override + public int nextSetBit(int start, int upperBound) { + int res = nextSetBit(start); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } } } From 3ee85a46af3c223b7d02fd7b3f4a1748a234a5f5 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Thu, 1 Aug 2024 07:15:35 -0700 Subject: [PATCH 38/66] Move CHANGES entry for GH#13559 from 10.0 to 9.12 --- lucene/CHANGES.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d65b9f847eb..4f64a73dab6 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -108,8 +108,6 @@ API Changes * GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand) -* GITHUB#13559: Add BitSet#nextSetBit(int, int) to get the index of the first set bit in range. (Egor Potemkin) - * GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) @@ -266,6 +264,8 @@ API Changes facilitate customizing per-leaf behavior of search without requiring to override `search(LeafReaderContext[], Weight, Collector)` which requires overriding the entire loop across the leaves (Luca Cavanna) +* GITHUB#13559: Add BitSet#nextSetBit(int, int) to get the index of the first set bit in range. (Egor Potemkin) + New Features --------------------- From f14eb2b2c59e4f368c627a28b4569c211037613f Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Thu, 1 Aug 2024 10:22:56 -0400 Subject: [PATCH 39/66] HnswLock: access locks via hash and only use for concurrent indexing (#13581) hnswlock: hash locks and only use for concurrent indexing --- .../util/hnsw/HnswConcurrentMergeBuilder.java | 18 +++-- .../lucene/util/hnsw/HnswGraphBuilder.java | 16 +++-- .../org/apache/lucene/util/hnsw/HnswLock.java | 70 +++++++++++++++++++ .../lucene/util/hnsw/NeighborArray.java | 3 - 4 files changed, 92 insertions(+), 15 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java index 2da7f8492ba..fd1a98d08c0 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java @@ -40,6 +40,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { private final TaskExecutor taskExecutor; private final ConcurrentMergeWorker[] workers; + private final HnswLock hnswLock; private InfoStream infoStream = InfoStream.getDefault(); private boolean frozen; @@ -55,6 +56,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { this.taskExecutor = taskExecutor; AtomicInteger workProgress = new AtomicInteger(0); workers = new ConcurrentMergeWorker[numWorker]; + hnswLock = new HnswLock(hnsw); for (int i = 0; i < numWorker; i++) { workers[i] = new ConcurrentMergeWorker( @@ -63,6 +65,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { beamWidth, HnswGraphBuilder.randSeed, hnsw, + hnswLock, initializedNodes, workProgress); } @@ -140,6 +143,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { int beamWidth, long seed, OnHeapHnswGraph hnsw, + HnswLock hnswLock, BitSet initializedNodes, AtomicInteger workProgress) throws IOException { @@ -149,8 +153,9 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { beamWidth, seed, hnsw, + hnswLock, new MergeSearcher( - new NeighborQueue(beamWidth, true), new FixedBitSet(hnsw.maxNodeId() + 1))); + new NeighborQueue(beamWidth, true), hnswLock, new FixedBitSet(hnsw.maxNodeId() + 1))); this.workProgress = workProgress; this.initializedNodes = initializedNodes; } @@ -195,26 +200,25 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { * that concurrent modification of the graph will not impact the search */ private static class MergeSearcher extends HnswGraphSearcher { + private final HnswLock hnswLock; private int[] nodeBuffer; private int upto; private int size; - private MergeSearcher(NeighborQueue candidates, BitSet visited) { + private MergeSearcher(NeighborQueue candidates, HnswLock hnswLock, BitSet visited) { super(candidates, visited); + this.hnswLock = hnswLock; } @Override void graphSeek(HnswGraph graph, int level, int targetNode) { - NeighborArray neighborArray = ((OnHeapHnswGraph) graph).getNeighbors(level, targetNode); - neighborArray.rwlock.readLock().lock(); - try { + try (HnswLock.LockedRow rowLock = hnswLock.read(level, targetNode)) { + NeighborArray neighborArray = rowLock.row; if (nodeBuffer == null || nodeBuffer.length < neighborArray.size()) { nodeBuffer = new int[neighborArray.size()]; } size = neighborArray.size(); if (size >= 0) System.arraycopy(neighborArray.nodes(), 0, nodeBuffer, 0, size); - } finally { - neighborArray.rwlock.readLock().unlock(); } upto = -1; } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index 2677c327642..6d06c1298a9 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -63,6 +63,7 @@ public class HnswGraphBuilder implements HnswBuilder { beamCandidates; // for levels of graph where we add the node protected final OnHeapHnswGraph hnsw; + protected final HnswLock hnswLock; private InfoStream infoStream = InfoStream.getDefault(); private boolean frozen; @@ -110,6 +111,7 @@ public class HnswGraphBuilder implements HnswBuilder { beamWidth, seed, hnsw, + null, new HnswGraphSearcher(new NeighborQueue(beamWidth, true), new FixedBitSet(hnsw.size()))); } @@ -131,6 +133,7 @@ public class HnswGraphBuilder implements HnswBuilder { int beamWidth, long seed, OnHeapHnswGraph hnsw, + HnswLock hnswLock, HnswGraphSearcher graphSearcher) throws IOException { if (M <= 0) { @@ -146,6 +149,7 @@ public class HnswGraphBuilder implements HnswBuilder { this.ml = M == 1 ? 1 : 1 / Math.log(1.0 * M); this.random = new SplittableRandom(seed); this.hnsw = hnsw; + this.hnswLock = hnswLock; this.graphSearcher = graphSearcher; entryCandidates = new GraphBuilderKnnCollector(1); beamCandidates = new GraphBuilderKnnCollector(beamWidth); @@ -327,12 +331,14 @@ public class HnswGraphBuilder implements HnswBuilder { continue; } int nbr = candidates.nodes()[i]; - NeighborArray nbrsOfNbr = hnsw.getNeighbors(level, nbr); - nbrsOfNbr.rwlock.writeLock().lock(); - try { + if (hnswLock != null) { + try (HnswLock.LockedRow rowLock = hnswLock.write(level, nbr)) { + NeighborArray nbrsOfNbr = rowLock.row; + nbrsOfNbr.addAndEnsureDiversity(node, candidates.scores()[i], nbr, scorerSupplier); + } + } else { + NeighborArray nbrsOfNbr = hnsw.getNeighbors(level, nbr); nbrsOfNbr.addAndEnsureDiversity(node, candidates.scores()[i], nbr, scorerSupplier); - } finally { - nbrsOfNbr.rwlock.writeLock().unlock(); } } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java new file mode 100644 index 00000000000..6dbfd83b566 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java @@ -0,0 +1,70 @@ +/* + * 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.hnsw; + +import java.io.Closeable; +import java.util.Objects; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Provide (read-and-write) locked access to rows of an OnHeapHnswGraph. For use by + * HnswConcurrentMerger and its HnswGraphBuilders. + */ +public class HnswLock { + private static final int NUM_LOCKS = 512; + private final ReentrantReadWriteLock[] locks; + private final OnHeapHnswGraph graph; + + HnswLock(OnHeapHnswGraph graph) { + this.graph = graph; + locks = new ReentrantReadWriteLock[NUM_LOCKS]; + for (int i = 0; i < NUM_LOCKS; i++) { + locks[i] = new ReentrantReadWriteLock(); + } + } + + LockedRow read(int level, int node) { + int lockid = Objects.hash(level, node) % NUM_LOCKS; + Lock lock = locks[lockid].readLock(); + lock.lock(); + return new LockedRow(graph.getNeighbors(level, node), lock); + } + + LockedRow write(int level, int node) { + int lockid = Objects.hash(level, node) % NUM_LOCKS; + Lock lock = locks[lockid].writeLock(); + lock.lock(); + return new LockedRow(graph.getNeighbors(level, node), lock); + } + + static class LockedRow implements Closeable { + final Lock lock; + final NeighborArray row; + + LockedRow(NeighborArray row, Lock lock) { + this.lock = lock; + this.row = row; + } + + @Override + public void close() { + lock.unlock(); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java index 7d1ed069c29..716364a39dc 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java @@ -19,8 +19,6 @@ package org.apache.lucene.util.hnsw; import java.io.IOException; import java.util.Arrays; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.lucene.util.ArrayUtil; /** @@ -37,7 +35,6 @@ public class NeighborArray { private final float[] scores; private final int[] nodes; private int sortedNodeSize; - public final ReadWriteLock rwlock = new ReentrantReadWriteLock(true); public NeighborArray(int maxSize, boolean descOrder) { nodes = new int[maxSize]; From 44a91338247611e3b8374cebdd01072f70375ca7 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 1 Aug 2024 16:24:16 +0200 Subject: [PATCH 40/66] Fix lazy decoding of freqs. (#13585) --- .../apache/lucene/codecs/lucene912/Lucene912PostingsReader.java | 1 + 1 file changed, 1 insertion(+) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index 6d1336d3750..5e66a200929 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -447,6 +447,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (freqFP != -1) { docIn.seek(freqFP); pforUtil.decode(docIn, freqBuffer); + freqFP = -1; } return (int) freqBuffer[docBufferUpto - 1]; From 2f297b77351a2b072e3c51911aacbea6a0f08ed2 Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Thu, 1 Aug 2024 10:30:28 -0400 Subject: [PATCH 41/66] Add CHANGES entry for GH-13581 --- lucene/CHANGES.txt | 2 ++ lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 4f64a73dab6..9d89c2365f1 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -326,6 +326,8 @@ Optimizations performance for queries that need skipping such as conjunctions. (Adrien Grand) +* GITHUB#13581: OnHeapHnswGraph no longer allocates a lock for every graph node (Mike Sokolov) + Changes in runtime behavior --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java index 6dbfd83b566..d7947c7fab1 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswLock.java @@ -26,7 +26,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * Provide (read-and-write) locked access to rows of an OnHeapHnswGraph. For use by * HnswConcurrentMerger and its HnswGraphBuilders. */ -public class HnswLock { +class HnswLock { private static final int NUM_LOCKS = 512; private final ReentrantReadWriteLock[] locks; private final OnHeapHnswGraph graph; From 836c8e76ad15fcd2894df215a21c8cbb293062a9 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 1 Aug 2024 15:05:05 -0400 Subject: [PATCH 42/66] Revert cosine deprecation (#13613) Opening for more pointed discussion. See latest discussion here: #13281 I was hoping to have a full answer for folks who use byte models by Lucene 10, but I just don't have that. I still want to remove the internal cosine optimized methods. We can do this if we store magnitudes along side the raw vectors. This way we can remove all the internal optimized cosine code as its complicated. --- lucene/CHANGES.txt | 4 +--- .../apache/lucene/index/VectorSimilarityFunction.java | 3 --- .../core/src/java/org/apache/lucene/util/VectorUtil.java | 9 +-------- 3 files changed, 2 insertions(+), 14 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 9d89c2365f1..6107eb173f6 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -254,9 +254,7 @@ Other API Changes --------------------- -* GITHUB#13281: Mark COSINE VectorSimilarityFunction as deprecated. (Pulkit Gupta) - -* GITHUB#13469: Expose FlatVectorsFormat as a first-class format; can be configured using a custom Codec. (Michael Sokolov) +* GITHUB#13469: Expose FlatVectorsFormat as a first-class format; can be configured using a custom Codec. (Michael Sokolov) * GITHUB#13612: Hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions. (Peter Gromov) diff --git a/lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java b/lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java index b625f0740a5..23bd0fd0ec6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java +++ b/lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java @@ -66,10 +66,7 @@ public enum VectorSimilarityFunction { * vectors to unit length, and instead use {@link VectorSimilarityFunction#DOT_PRODUCT}. You * should only use this function if you need to preserve the original vectors and cannot normalize * them in advance. The similarity score is normalised to assure it is positive. - * - * @deprecated Use MAXIMUM_INNER_PRODUCT or DOT_PRODUCT instead */ - @Deprecated COSINE { @Override public float compare(float[] v1, float[] v2) { diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index 0ae563c8701..f122ae95544 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java @@ -70,9 +70,7 @@ public final class VectorUtil { * Returns the cosine similarity between the two vectors. * * @throws IllegalArgumentException if the vectors' dimensions differ. - * @deprecated use dot-product instead using normalized vectors */ - @Deprecated public static float cosine(float[] a, float[] b) { if (a.length != b.length) { throw new IllegalArgumentException("vector dimensions differ: " + a.length + "!=" + b.length); @@ -82,12 +80,7 @@ public final class VectorUtil { return r; } - /** - * Returns the cosine similarity between the two vectors. - * - * @deprecated use dot-product instead using normalized vectors - */ - @Deprecated + /** Returns the cosine similarity between the two vectors. */ public static float cosine(byte[] a, byte[] b) { if (a.length != b.length) { throw new IllegalArgumentException("vector dimensions differ: " + a.length + "!=" + b.length); From 99f35e3f8b76430530d769951e49c80ed5959b7f Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Thu, 1 Aug 2024 19:16:39 -0700 Subject: [PATCH 43/66] Remove some BitSet#nextSetBit code duplication (#13625) --- .../java/org/apache/lucene/util/BitSet.java | 6 ++- .../org/apache/lucene/util/FixedBitSet.java | 38 ++++++-------- .../apache/lucene/util/SparseFixedBitSet.java | 50 +++++++------------ .../lucene/tests/util/BaseBitSetTestCase.java | 12 ++--- 4 files changed, 43 insertions(+), 63 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/BitSet.java b/lucene/core/src/java/org/apache/lucene/util/BitSet.java index 4946957caf1..347e05d2e52 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/BitSet.java @@ -90,7 +90,11 @@ public abstract class BitSet implements Bits, Accountable { * Returns the index of the first set bit starting at the index specified. {@link * DocIdSetIterator#NO_MORE_DOCS} is returned if there are no more set bits. */ - public abstract int nextSetBit(int index); + public int nextSetBit(int index) { + // Default implementation. Subclasses may be able to override with a more performant + // implementation. + return nextSetBit(index, length()); + } /** * Returns the index of the first set bit from start (inclusive) until end (exclusive). {@link diff --git a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java index 750c6684e95..5d1f394853c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java @@ -272,27 +272,23 @@ public final class FixedBitSet extends BitSet { @Override public int nextSetBit(int index) { - // Depends on the ghost bits being clear! - assert index >= 0 && index < numBits : "index=" + index + ", numBits=" + numBits; - int i = index >> 6; - long word = bits[i] >> index; // skip all the bits to the right of index - - if (word != 0) { - return index + Long.numberOfTrailingZeros(word); - } - - while (++i < numWords) { - word = bits[i]; - if (word != 0) { - return (i << 6) + Long.numberOfTrailingZeros(word); - } - } - - return DocIdSetIterator.NO_MORE_DOCS; + // Override with a version that skips the bound check on the result since we know it will not + // go OOB: + return nextSetBitInRange(index, numBits); } @Override public int nextSetBit(int start, int upperBound) { + int res = nextSetBitInRange(start, upperBound); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + + /** + * Returns the next set bit in the specified range, but treats `upperBound` as a best-effort hint + * rather than a hard requirement. Note that this may return a result that is >= upperBound in + * some cases, so callers must add their own check if `upperBound` is a hard requirement. + */ + private int nextSetBitInRange(int start, int upperBound) { // Depends on the ghost bits being clear! assert start >= 0 && start < numBits : "index=" + start + ", numBits=" + numBits; assert start < upperBound : "index=" + start + ", upperBound=" + upperBound; @@ -301,16 +297,14 @@ public final class FixedBitSet extends BitSet { long word = bits[i] >> start; // skip all the bits to the right of index if (word != 0) { - int res = start + Long.numberOfTrailingZeros(word); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + return start + Long.numberOfTrailingZeros(word); } - int limit = bits2words(upperBound); + int limit = upperBound == numBits ? numWords : bits2words(upperBound); while (++i < limit) { word = bits[i]; if (word != 0) { - int res = (i << 6) + Long.numberOfTrailingZeros(word); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + return (i << 6) + Long.numberOfTrailingZeros(word); } } diff --git a/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java b/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java index 66805d53d94..3104948cc4c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java @@ -337,36 +337,26 @@ public class SparseFixedBitSet extends BitSet { @Override public int nextSetBit(int i) { - assert i < length; - final int i4096 = i >>> 12; - final long index = indices[i4096]; - final long[] bitArray = this.bits[i4096]; - int i64 = i >>> 6; - int o = Long.bitCount(index & ((1L << i64) - 1)); - if ((index & (1L << i64)) != 0) { - // There is at least one bit that is set in the current long, check if - // one of them is after i - final long bits = bitArray[o] >>> i; // shifts are mod 64 - if (bits != 0) { - return i + Long.numberOfTrailingZeros(bits); - } - o += 1; - } - final long indexBits = index >>> i64 >>> 1; - if (indexBits == 0) { - // no more bits are set in the current block of 4096 bits, go to the next one - return firstDoc(i4096 + 1, indices.length); - } - // there are still set bits - i64 += 1 + Long.numberOfTrailingZeros(indexBits); - final long bits = bitArray[o]; - return (i64 << 6) | Long.numberOfTrailingZeros(bits); + // Override with a version that skips the bound check on the result since we know it will not + // go OOB: + return nextSetBitInRange(i, length); } @Override public int nextSetBit(int start, int upperBound) { + int res = nextSetBitInRange(start, upperBound); + return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + } + + /** + * Returns the next set bit in the specified range, but treats `upperBound` as a best-effort hint + * rather than a hard requirement. Note that this may return a result that is >= upperBound in + * some cases, so callers must add their own check if `upperBound` is a hard requirement. + */ + private int nextSetBitInRange(int start, int upperBound) { assert start < length; - assert upperBound > start; + assert upperBound > start && upperBound <= length + : "upperBound=" + upperBound + ", start=" + start + ", length=" + length; final int i4096 = start >>> 12; final long index = indices[i4096]; final long[] bitArray = this.bits[i4096]; @@ -378,22 +368,20 @@ public class SparseFixedBitSet extends BitSet { // one of them is after i final long bits = bitArray[o] >>> start; // shifts are mod 64 if (bits != 0) { - int res = start + Long.numberOfTrailingZeros(bits); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + return start + Long.numberOfTrailingZeros(bits); } o += 1; } final long indexBits = index >>> i64 >>> 1; if (indexBits == 0) { // no more bits are set in the current block of 4096 bits, go to the next one - int res = firstDoc(i4096 + 1, blockCount(upperBound)); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + int i4096upper = upperBound == length ? indices.length : blockCount(upperBound); + return firstDoc(i4096 + 1, i4096upper); } // there are still set bits i64 += 1 + Long.numberOfTrailingZeros(indexBits); final long bits = bitArray[o]; - int res = (i64 << 6) | Long.numberOfTrailingZeros(bits); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; + return (i64 << 6) | Long.numberOfTrailingZeros(bits); } /** Return the last document that occurs on or before the provided block index. */ diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java index 095bcabbe4c..205d8861322 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/BaseBitSetTestCase.java @@ -341,18 +341,12 @@ public abstract class BaseBitSetTestCase extends LuceneTestCas } @Override - public int nextSetBit(int i) { - int next = bitSet.nextSetBit(i); - if (next == -1) { + public int nextSetBit(int start, int upperBound) { + int next = bitSet.nextSetBit(start); + if (next == -1 || next >= upperBound) { next = DocIdSetIterator.NO_MORE_DOCS; } return next; } - - @Override - public int nextSetBit(int start, int upperBound) { - int res = nextSetBit(start); - return res < upperBound ? res : DocIdSetIterator.NO_MORE_DOCS; - } } } From 35a5d7323b8b13672ad942907adf40aa2aec3a17 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Thu, 1 Aug 2024 19:19:20 -0700 Subject: [PATCH 44/66] CHANGES entry for GH#13625 --- lucene/CHANGES.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 6107eb173f6..0c3aa40c5c2 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -284,6 +284,8 @@ Improvements * GITHUB#13562: Add Intervals.regexp and Intervals.range methods to produce IntervalsSource for regexp and range queries. (Mayya Sharipova) +* GITHUB#13625: Remove BitSet#nextSetBit code duplication. (Greg Miller) + Optimizations --------------------- From ad2f02c013d4c634b0dadf57282fc13fb71f9fcc Mon Sep 17 00:00:00 2001 From: Zhang Chao <80152403@qq.com> Date: Fri, 2 Aug 2024 15:28:03 +0800 Subject: [PATCH 45/66] Fix testAddDocumentOnDiskFull to handle IllegalStateException from IndexWriter#close (#13558) --- .../test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java index 78f59300e32..a9a9254badc 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java @@ -111,7 +111,7 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase { System.out.println("TEST: now close"); } writer.close(); - } catch (IOException e) { + } catch (IOException | IllegalStateException e) { if (VERBOSE) { System.out.println("TEST: exception on close; retry w/ no disk space limit"); e.printStackTrace(System.out); From 1ce1156de441ff5be17b3f83f781faf10c774537 Mon Sep 17 00:00:00 2001 From: Jakub Slowinski <32519034+slow-J@users.noreply.github.com> Date: Fri, 2 Aug 2024 13:03:09 +0100 Subject: [PATCH 46/66] Fix failing unit test - TestTopDocsCollector#testResultsOrder (#13621) --- .../test/org/apache/lucene/search/TestTopDocsCollector.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java index 0566061f357..e91858800a2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java @@ -72,9 +72,6 @@ public class TestTopDocsCollector extends LuceneTestCase { } private static final class MyTopDocsCollector extends TopDocsCollector { - - private int idx = 0; - public MyTopDocsCollector(int size) { super(new HitQueue(size, false)); } @@ -92,11 +89,12 @@ public class TestTopDocsCollector extends LuceneTestCase { public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { final int base = context.docBase; return new LeafCollector() { + private int idx = 0; @Override public void collect(int doc) { ++totalHits; - pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++])); + pq.insertWithOverflow(new ScoreDoc(doc + base, scores[context.docBase + idx++])); } @Override From cb8bc757507952662ee4f0a7713b9b3d7be80eb9 Mon Sep 17 00:00:00 2001 From: zhouhui Date: Fri, 2 Aug 2024 23:12:34 +0800 Subject: [PATCH 47/66] Rename prefix to prefixLength, suffix to suffixLength. (#13629) --- .../lucene90/blocktree/SegmentTermsEnum.java | 28 +++++---- .../blocktree/SegmentTermsEnumFrame.java | 61 ++++++++++--------- .../codecs/lucene90/blocktree/Stats.java | 6 +- 3 files changed, 49 insertions(+), 46 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java index 91776585407..45ec4ee06ba 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java @@ -277,10 +277,10 @@ final class SegmentTermsEnum extends BaseTermsEnum { // System.out.println(" skip rewind!"); // } } - assert length == f.prefix; + assert length == f.prefixLength; } else { f.nextEnt = -1; - f.prefix = length; + f.prefixLength = length; f.state.termBlockOrd = 0; f.fpOrig = f.fp = fp; f.lastSubFP = -1; @@ -488,7 +488,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { // toHex(targetLabel)); // } - validIndexPrefix = currentFrame.prefix; + validIndexPrefix = currentFrame.prefixLength; // validIndexPrefix = targetUpto; currentFrame.scanToFloorFrame(target); @@ -550,7 +550,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { } // validIndexPrefix = targetUpto; - validIndexPrefix = currentFrame.prefix; + validIndexPrefix = currentFrame.prefixLength; currentFrame.scanToFloorFrame(target); @@ -772,7 +772,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { // targetLabel); // } - validIndexPrefix = currentFrame.prefix; + validIndexPrefix = currentFrame.prefixLength; // validIndexPrefix = targetUpto; currentFrame.scanToFloorFrame(target); @@ -830,7 +830,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { } // validIndexPrefix = targetUpto; - validIndexPrefix = currentFrame.prefix; + validIndexPrefix = currentFrame.prefixLength; currentFrame.scanToFloorFrame(target); @@ -868,7 +868,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { while (true) { SegmentTermsEnumFrame f = getFrame(ord); assert f != null; - final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefix); + final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefixLength); if (f.nextEnt == -1) { out.println( " frame " @@ -879,7 +879,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" - + f.prefix + + f.prefixLength + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) @@ -907,7 +907,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" - + f.prefix + + f.prefixLength + " prefix=" + prefix + " nextEnt=" @@ -932,12 +932,14 @@ final class SegmentTermsEnum extends BaseTermsEnum { } if (fr.index != null) { assert !isSeekFrame || f.arc != null : "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc; - if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix - 1) & 0xFF)) { + if (f.prefixLength > 0 + && isSeekFrame + && f.arc.label() != (term.byteAt(f.prefixLength - 1) & 0xFF)) { out.println( " broken seek state: arc.label=" + (char) f.arc.label() + " vs term byte=" - + (char) (term.byteAt(f.prefix - 1) & 0xFF)); + + (char) (term.byteAt(f.prefixLength - 1) & 0xFF)); throw new RuntimeException("seek state is broken"); } BytesRef output = Util.get(fr.index, prefix); @@ -965,7 +967,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { if (f == currentFrame) { break; } - if (f.prefix == validIndexPrefix) { + if (f.prefixLength == validIndexPrefix) { isSeekFrame = false; } ord++; @@ -1046,7 +1048,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { // Note that the seek state (last seek) has been // invalidated beyond this depth - validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix); + validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefixLength); // if (DEBUG) { // System.out.println(" reset validIndexPrefix=" + validIndexPrefix); // } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java index 5ecbc3c173e..9988c45bdf7 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java @@ -59,7 +59,7 @@ final class SegmentTermsEnumFrame { final ByteArrayDataInput floorDataReader = new ByteArrayDataInput(); // Length of prefix shared by all terms in this block - int prefix; + int prefixLength; // Number of entries (term or sub-block) in this block int entCount; @@ -318,11 +318,11 @@ final class SegmentTermsEnumFrame { assert nextEnt != -1 && nextEnt < entCount : "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp; nextEnt++; - suffix = suffixLengthsReader.readVInt(); + suffixLength = suffixLengthsReader.readVInt(); startBytePos = suffixesReader.getPosition(); - ste.term.setLength(prefix + suffix); + ste.term.setLength(prefixLength + suffixLength); ste.term.grow(ste.term.length()); - suffixesReader.readBytes(ste.term.bytes(), prefix, suffix); + suffixesReader.readBytes(ste.term.bytes(), prefixLength, suffixLength); ste.termExists = true; } @@ -346,11 +346,11 @@ final class SegmentTermsEnumFrame { : "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp; nextEnt++; final int code = suffixLengthsReader.readVInt(); - suffix = code >>> 1; + suffixLength = code >>> 1; startBytePos = suffixesReader.getPosition(); - ste.term.setLength(prefix + suffix); + ste.term.setLength(prefixLength + suffixLength); ste.term.grow(ste.term.length()); - suffixesReader.readBytes(ste.term.bytes(), prefix, suffix); + suffixesReader.readBytes(ste.term.bytes(), prefixLength, suffixLength); if ((code & 1) == 0) { // A normal term ste.termExists = true; @@ -375,7 +375,7 @@ final class SegmentTermsEnumFrame { // floor blocks we "typically" get public void scanToFloorFrame(BytesRef target) { - if (!isFloor || target.length <= prefix) { + if (!isFloor || target.length <= prefixLength) { // if (DEBUG) { // System.out.println(" scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" + // target.length + " vs prefix=" + prefix); @@ -383,7 +383,7 @@ final class SegmentTermsEnumFrame { return; } - final int targetLabel = target.bytes[target.offset + prefix] & 0xFF; + final int targetLabel = target.bytes[target.offset + prefixLength] & 0xFF; // if (DEBUG) { // System.out.println(" scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" + @@ -497,7 +497,7 @@ final class SegmentTermsEnumFrame { // Used only by assert private boolean prefixMatches(BytesRef target) { - for (int bytePos = 0; bytePos < prefix; bytePos++) { + for (int bytePos = 0; bytePos < prefixLength; bytePos++) { if (target.bytes[target.offset + bytePos] != ste.term.byteAt(bytePos)) { return false; } @@ -553,7 +553,7 @@ final class SegmentTermsEnumFrame { } private int startBytePos; - private int suffix; + private int suffixLength; private long subCode; CompressionAlgorithm compressionAlg = CompressionAlgorithm.NO_COMPRESSION; @@ -584,7 +584,7 @@ final class SegmentTermsEnumFrame { do { nextEnt++; - suffix = suffixLengthsReader.readVInt(); + suffixLength = suffixLengthsReader.readVInt(); // if (DEBUG) { // BytesRef suffixBytesRef = new BytesRef(); @@ -596,16 +596,16 @@ final class SegmentTermsEnumFrame { // } startBytePos = suffixesReader.getPosition(); - suffixesReader.skipBytes(suffix); + suffixesReader.skipBytes(suffixLength); // Loop over bytes in the suffix, comparing to the target final int cmp = Arrays.compareUnsigned( suffixBytes, startBytePos, - startBytePos + suffix, + startBytePos + suffixLength, target.bytes, - target.offset + prefix, + target.offset + prefixLength, target.offset + target.length); if (cmp < 0) { @@ -674,7 +674,7 @@ final class SegmentTermsEnumFrame { assert prefixMatches(target); - suffix = suffixLengthsReader.readVInt(); + suffixLength = suffixLengthsReader.readVInt(); // TODO early terminate when target length unequals suffix + prefix. // But we need to keep the same status with scanToTermLeaf. int start = nextEnt; @@ -684,16 +684,16 @@ final class SegmentTermsEnumFrame { while (start <= end) { int mid = (start + end) >>> 1; nextEnt = mid + 1; - startBytePos = mid * suffix; + startBytePos = mid * suffixLength; // Binary search bytes in the suffix, comparing to the target. cmp = Arrays.compareUnsigned( suffixBytes, startBytePos, - startBytePos + suffix, + startBytePos + suffixLength, target.bytes, - target.offset + prefix, + target.offset + prefixLength, target.offset + target.length); if (cmp < 0) { start = mid + 1; @@ -701,7 +701,7 @@ final class SegmentTermsEnumFrame { end = mid - 1; } else { // Exact match! - suffixesReader.setPosition(startBytePos + suffix); + suffixesReader.setPosition(startBytePos + suffixLength); fillTerm(); // if (DEBUG) System.out.println(" found!"); return SeekStatus.FOUND; @@ -724,14 +724,14 @@ final class SegmentTermsEnumFrame { // If binary search ended at the less term, and greater term exists. // We need to advance to the greater term. if (cmp < 0) { - startBytePos += suffix; + startBytePos += suffixLength; nextEnt++; } - suffixesReader.setPosition(startBytePos + suffix); + suffixesReader.setPosition(startBytePos + suffixLength); fillTerm(); } else { seekStatus = SeekStatus.END; - suffixesReader.setPosition(startBytePos + suffix); + suffixesReader.setPosition(startBytePos + suffixLength); if (exactOnly) { fillTerm(); } @@ -769,7 +769,7 @@ final class SegmentTermsEnumFrame { nextEnt++; final int code = suffixLengthsReader.readVInt(); - suffix = code >>> 1; + suffixLength = code >>> 1; // if (DEBUG) { // BytesRef suffixBytesRef = new BytesRef(); @@ -782,7 +782,7 @@ final class SegmentTermsEnumFrame { // } startBytePos = suffixesReader.getPosition(); - suffixesReader.skipBytes(suffix); + suffixesReader.skipBytes(suffixLength); ste.termExists = (code & 1) == 0; if (ste.termExists) { state.termBlockOrd++; @@ -796,9 +796,9 @@ final class SegmentTermsEnumFrame { Arrays.compareUnsigned( suffixBytes, startBytePos, - startBytePos + suffix, + startBytePos + suffixLength, target.bytes, - target.offset + prefix, + target.offset + prefixLength, target.offset + target.length); if (cmp < 0) { @@ -819,7 +819,8 @@ final class SegmentTermsEnumFrame { // us to position to the next term after // the target, so we must recurse into the // sub-frame(s): - ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, prefix + suffix); + ste.currentFrame = + ste.pushFrame(null, ste.currentFrame.lastSubFP, prefixLength + suffixLength); ste.currentFrame.loadBlock(); while (ste.currentFrame.next()) { ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length()); @@ -864,9 +865,9 @@ final class SegmentTermsEnumFrame { } private void fillTerm() { - final int termLength = prefix + suffix; + final int termLength = prefixLength + suffixLength; ste.term.setLength(termLength); ste.term.grow(termLength); - System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix); + System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefixLength, suffixLength); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java index e8f25197074..ceeef4a8687 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java @@ -116,10 +116,10 @@ public class Stats { nonFloorBlockCount++; } - if (blockCountByPrefixLen.length <= frame.prefix) { - blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1 + frame.prefix); + if (blockCountByPrefixLen.length <= frame.prefixLength) { + blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1 + frame.prefixLength); } - blockCountByPrefixLen[frame.prefix]++; + blockCountByPrefixLen[frame.prefixLength]++; startBlockCount++; totalBlockSuffixBytes += frame.totalSuffixBytes; totalUncompressedBlockSuffixBytes += frame.suffixesReader.length(); From 26b46ced0714551bb68ee0ccad513e232492c942 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Fri, 2 Aug 2024 11:59:16 -0400 Subject: [PATCH 48/66] KMeans clustering algorithm (#13604) Implement Kmeans clustering algorithm for vectors. Knn algorithms that further reduce memory usage of vectors (such as Product Quantization, RaBitQ etc) require clustering of vectors. This implements KMeans clustering algorithm. Co-authored-by: Jim Ferenczi jim.ferenczi@elastic.co --- lucene/CHANGES.txt | 2 + lucene/sandbox/src/java/module-info.java | 1 + .../sandbox/codecs/quantization/KMeans.java | 394 ++++++++++++++++++ .../codecs/quantization/SampleReader.java | 134 ++++++ .../codecs/quantization/package-info.java | 19 + .../codecs/quantization/TestKMeans.java | 142 +++++++ 6 files changed, 692 insertions(+) create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/KMeans.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/SampleReader.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/package-info.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/quantization/TestKMeans.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 0c3aa40c5c2..fb91b89a2a4 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -137,6 +137,8 @@ New Features * GITHUB#13597: Align doc value skipper interval boundaries when an interval contains a constant value. (Ignacio Vera) +* GITHUB#13604: Add Kmeans clustering on vectors (Mayya Sharipova, Jim Ferenczi, Tom Veasey) + Improvements --------------------- diff --git a/lucene/sandbox/src/java/module-info.java b/lucene/sandbox/src/java/module-info.java index c51a25691ef..3daace50cee 100644 --- a/lucene/sandbox/src/java/module-info.java +++ b/lucene/sandbox/src/java/module-info.java @@ -22,6 +22,7 @@ module org.apache.lucene.sandbox { exports org.apache.lucene.payloads; exports org.apache.lucene.sandbox.codecs.idversion; + exports org.apache.lucene.sandbox.codecs.quantization; exports org.apache.lucene.sandbox.document; exports org.apache.lucene.sandbox.queries; exports org.apache.lucene.sandbox.search; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/KMeans.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/KMeans.java new file mode 100644 index 00000000000..bb9d3ca63df --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/KMeans.java @@ -0,0 +1,394 @@ +/* + * 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.sandbox.codecs.quantization; + +import static org.apache.lucene.sandbox.codecs.quantization.SampleReader.createSampleReader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.VectorUtil; +import org.apache.lucene.util.hnsw.NeighborQueue; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; + +/** KMeans clustering algorithm for vectors */ +public class KMeans { + public static final int MAX_NUM_CENTROIDS = Short.MAX_VALUE; // 32767 + public static final int DEFAULT_RESTARTS = 5; + public static final int DEFAULT_ITRS = 10; + public static final int DEFAULT_SAMPLE_SIZE = 100_000; + + private final RandomAccessVectorValues.Floats vectors; + private final int numVectors; + private final int numCentroids; + private final Random random; + private final KmeansInitializationMethod initializationMethod; + private final int restarts; + private final int iters; + + /** + * Cluster vectors into a given number of clusters + * + * @param vectors float vectors + * @param similarityFunction vector similarity function. For COSINE similarity, vectors must be + * normalized. + * @param numClusters number of cluster to cluster vector into + * @return results of clustering: produced centroids and for each vector its centroid + * @throws IOException when if there is an error accessing vectors + */ + public static Results cluster( + RandomAccessVectorValues.Floats vectors, + VectorSimilarityFunction similarityFunction, + int numClusters) + throws IOException { + return cluster( + vectors, + numClusters, + true, + 42L, + KmeansInitializationMethod.PLUS_PLUS, + similarityFunction == VectorSimilarityFunction.COSINE, + DEFAULT_RESTARTS, + DEFAULT_ITRS, + DEFAULT_SAMPLE_SIZE); + } + + /** + * Expert: Cluster vectors into a given number of clusters + * + * @param vectors float vectors + * @param numClusters number of cluster to cluster vector into + * @param assignCentroidsToVectors if {@code true} assign centroids for all vectors. Centroids are + * computed on a sample of vectors. If this parameter is {@code true}, in results also return + * for all vectors what centroids they belong to. + * @param seed random seed + * @param initializationMethod Kmeans initialization method + * @param normalizeCenters for cosine distance, set to true, to use spherical k-means where + * centers are normalized + * @param restarts how many times to run Kmeans algorithm + * @param iters how many iterations to do within a single run + * @param sampleSize sample size to select from all vectors on which to run Kmeans algorithm + * @return results of clustering: produced centroids and if {@code assignCentroidsToVectors == + * true} also for each vector its centroid + * @throws IOException if there is error accessing vectors + */ + public static Results cluster( + RandomAccessVectorValues.Floats vectors, + int numClusters, + boolean assignCentroidsToVectors, + long seed, + KmeansInitializationMethod initializationMethod, + boolean normalizeCenters, + int restarts, + int iters, + int sampleSize) + throws IOException { + if (vectors.size() == 0) { + return null; + } + if (numClusters < 1 || numClusters > MAX_NUM_CENTROIDS) { + throw new IllegalArgumentException( + "[numClusters] must be between [1] and [" + MAX_NUM_CENTROIDS + "]"); + } + // adjust sampleSize and numClusters + sampleSize = Math.max(sampleSize, 100 * numClusters); + if (sampleSize > vectors.size()) { + sampleSize = vectors.size(); + // Decrease the number of clusters if needed + int maxNumClusters = Math.max(1, sampleSize / 100); + numClusters = Math.min(numClusters, maxNumClusters); + } + + Random random = new Random(seed); + float[][] centroids; + if (numClusters == 1) { + centroids = new float[1][vectors.dimension()]; + } else { + RandomAccessVectorValues.Floats sampleVectors = + vectors.size() <= sampleSize ? vectors : createSampleReader(vectors, sampleSize, seed); + KMeans kmeans = + new KMeans(sampleVectors, numClusters, random, initializationMethod, restarts, iters); + centroids = kmeans.computeCentroids(normalizeCenters); + } + + short[] vectorCentroids = null; + // Assign each vector to the nearest centroid and update the centres + if (assignCentroidsToVectors) { + vectorCentroids = new short[vectors.size()]; + // Use kahan summation to get more precise results + KMeans.runKMeansStep(vectors, centroids, vectorCentroids, true, normalizeCenters); + } + return new Results(centroids, vectorCentroids); + } + + private KMeans( + RandomAccessVectorValues.Floats vectors, + int numCentroids, + Random random, + KmeansInitializationMethod initializationMethod, + int restarts, + int iters) { + this.vectors = vectors; + this.numVectors = vectors.size(); + this.numCentroids = numCentroids; + this.random = random; + this.initializationMethod = initializationMethod; + this.restarts = restarts; + this.iters = iters; + } + + private float[][] computeCentroids(boolean normalizeCenters) throws IOException { + short[] vectorCentroids = new short[numVectors]; + double minSquaredDist = Double.MAX_VALUE; + double squaredDist = 0; + float[][] bestCentroids = null; + + for (int restart = 0; restart < restarts; restart++) { + float[][] centroids = + switch (initializationMethod) { + case FORGY -> initializeForgy(); + case RESERVOIR_SAMPLING -> initializeReservoirSampling(); + case PLUS_PLUS -> initializePlusPlus(); + }; + double prevSquaredDist = Double.MAX_VALUE; + for (int iter = 0; iter < iters; iter++) { + squaredDist = runKMeansStep(vectors, centroids, vectorCentroids, false, normalizeCenters); + // Check for convergence + if (prevSquaredDist <= (squaredDist + 1e-6)) { + break; + } + prevSquaredDist = squaredDist; + } + if (squaredDist < minSquaredDist) { + minSquaredDist = squaredDist; + bestCentroids = centroids; + } + } + return bestCentroids; + } + + /** + * Initialize centroids using Forgy method: randomly select numCentroids vectors for initial + * centroids + */ + private float[][] initializeForgy() throws IOException { + Set selection = new HashSet<>(); + while (selection.size() < numCentroids) { + selection.add(random.nextInt(numVectors)); + } + float[][] initialCentroids = new float[numCentroids][]; + int i = 0; + for (Integer selectedIdx : selection) { + float[] vector = vectors.vectorValue(selectedIdx); + initialCentroids[i++] = ArrayUtil.copyOfSubArray(vector, 0, vector.length); + } + return initialCentroids; + } + + /** Initialize centroids using a reservoir sampling method */ + private float[][] initializeReservoirSampling() throws IOException { + float[][] initialCentroids = new float[numCentroids][]; + for (int index = 0; index < numVectors; index++) { + float[] vector = vectors.vectorValue(index); + if (index < numCentroids) { + initialCentroids[index] = ArrayUtil.copyOfSubArray(vector, 0, vector.length); + } else if (random.nextDouble() < numCentroids * (1.0 / index)) { + int c = random.nextInt(numCentroids); + initialCentroids[c] = ArrayUtil.copyOfSubArray(vector, 0, vector.length); + } + } + return initialCentroids; + } + + /** Initialize centroids using Kmeans++ method */ + private float[][] initializePlusPlus() throws IOException { + float[][] initialCentroids = new float[numCentroids][]; + // Choose the first centroid uniformly at random + int firstIndex = random.nextInt(numVectors); + float[] value = vectors.vectorValue(firstIndex); + initialCentroids[0] = ArrayUtil.copyOfSubArray(value, 0, value.length); + + // Store distances of each point to the nearest centroid + float[] minDistances = new float[numVectors]; + Arrays.fill(minDistances, Float.MAX_VALUE); + + // Step 2 and 3: Select remaining centroids + for (int i = 1; i < numCentroids; i++) { + // Update distances with the new centroid + double totalSum = 0; + for (int j = 0; j < numVectors; j++) { + // TODO: replace with RandomVectorScorer::score possible on quantized vectors + float dist = VectorUtil.squareDistance(vectors.vectorValue(j), initialCentroids[i - 1]); + if (dist < minDistances[j]) { + minDistances[j] = dist; + } + totalSum += minDistances[j]; + } + + // Randomly select next centroid + double r = totalSum * random.nextDouble(); + double cumulativeSum = 0; + int nextCentroidIndex = -1; + for (int j = 0; j < numVectors; j++) { + cumulativeSum += minDistances[j]; + if (cumulativeSum >= r && minDistances[j] > 0) { + nextCentroidIndex = j; + break; + } + } + // Update centroid + value = vectors.vectorValue(nextCentroidIndex); + initialCentroids[i] = ArrayUtil.copyOfSubArray(value, 0, value.length); + } + return initialCentroids; + } + + /** + * Run kmeans step + * + * @param vectors float vectors + * @param centroids centroids, new calculated centroids are written here + * @param docCentroids for each document which centroid it belongs to, results will be written + * here + * @param useKahanSummation for large datasets use Kahan summation to calculate centroids, since + * we can easily reach the limits of float precision + * @param normalizeCentroids if centroids should be normalized; used for cosine similarity only + * @throws IOException if there is an error accessing vector values + */ + private static double runKMeansStep( + RandomAccessVectorValues.Floats vectors, + float[][] centroids, + short[] docCentroids, + boolean useKahanSummation, + boolean normalizeCentroids) + throws IOException { + short numCentroids = (short) centroids.length; + + float[][] newCentroids = new float[numCentroids][centroids[0].length]; + int[] newCentroidSize = new int[numCentroids]; + float[][] compensations = null; + if (useKahanSummation) { + compensations = new float[numCentroids][centroids[0].length]; + } + + double sumSquaredDist = 0; + for (int docID = 0; docID < vectors.size(); docID++) { + float[] vector = vectors.vectorValue(docID); + short bestCentroid = 0; + if (numCentroids > 1) { + float minSquaredDist = Float.MAX_VALUE; + for (short c = 0; c < numCentroids; c++) { + // TODO: replace with RandomVectorScorer::score possible on quantized vectors + float squareDist = VectorUtil.squareDistance(centroids[c], vector); + if (squareDist < minSquaredDist) { + bestCentroid = c; + minSquaredDist = squareDist; + } + } + sumSquaredDist += minSquaredDist; + } + + newCentroidSize[bestCentroid] += 1; + for (int dim = 0; dim < vector.length; dim++) { + if (useKahanSummation) { + float y = vector[dim] - compensations[bestCentroid][dim]; + float t = newCentroids[bestCentroid][dim] + y; + compensations[bestCentroid][dim] = (t - newCentroids[bestCentroid][dim]) - y; + newCentroids[bestCentroid][dim] = t; + } else { + newCentroids[bestCentroid][dim] += vector[dim]; + } + } + docCentroids[docID] = bestCentroid; + } + + List unassignedCentroids = new ArrayList<>(); + for (int c = 0; c < numCentroids; c++) { + if (newCentroidSize[c] > 0) { + for (int dim = 0; dim < newCentroids[c].length; dim++) { + centroids[c][dim] = newCentroids[c][dim] / newCentroidSize[c]; + } + } else { + unassignedCentroids.add(c); + } + } + if (unassignedCentroids.size() > 0) { + assignCentroids(vectors, centroids, unassignedCentroids); + } + if (normalizeCentroids) { + for (int c = 0; c < centroids.length; c++) { + VectorUtil.l2normalize(centroids[c], false); + } + } + return sumSquaredDist; + } + + /** + * For centroids that did not get any points, assign outlying points to them chose points by + * descending distance to the current centroid set + */ + static void assignCentroids( + RandomAccessVectorValues.Floats vectors, + float[][] centroids, + List unassignedCentroidsIdxs) + throws IOException { + int[] assignedCentroidsIdxs = new int[centroids.length - unassignedCentroidsIdxs.size()]; + int assignedIndex = 0; + for (int i = 0; i < centroids.length; i++) { + if (unassignedCentroidsIdxs.contains(i) == false) { + assignedCentroidsIdxs[assignedIndex++] = i; + } + } + NeighborQueue queue = new NeighborQueue(unassignedCentroidsIdxs.size(), false); + for (int i = 0; i < vectors.size(); i++) { + float[] vector = vectors.vectorValue(i); + for (short j = 0; j < assignedCentroidsIdxs.length; j++) { + float squareDist = VectorUtil.squareDistance(centroids[assignedCentroidsIdxs[j]], vector); + queue.insertWithOverflow(i, squareDist); + } + } + for (int i = 0; i < unassignedCentroidsIdxs.size(); i++) { + float[] vector = vectors.vectorValue(queue.topNode()); + int unassignedCentroidIdx = unassignedCentroidsIdxs.get(i); + centroids[unassignedCentroidIdx] = ArrayUtil.copyArray(vector); + queue.pop(); + } + } + + /** Kmeans initialization methods */ + public enum KmeansInitializationMethod { + FORGY, + RESERVOIR_SAMPLING, + PLUS_PLUS + } + + /** + * Results of KMeans clustering + * + * @param centroids the produced centroids + * @param vectorCentroids for each vector which centroid it belongs to, we use short type, as we + * expect less than {@code MAX_NUM_CENTROIDS} which is equal to 32767 centroids. Can be {@code + * null} if they were not computed. + */ + public record Results(float[][] centroids, short[] vectorCentroids) {} +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/SampleReader.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/SampleReader.java new file mode 100644 index 00000000000..9a718c81101 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/SampleReader.java @@ -0,0 +1,134 @@ +/* + * 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.sandbox.codecs.quantization; + +import java.io.IOException; +import java.util.Random; +import java.util.function.IntUnaryOperator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; + +/** A reader of vector values that samples a subset of the vectors. */ +public class SampleReader implements RandomAccessVectorValues.Floats { + private final RandomAccessVectorValues.Floats origin; + private final int sampleSize; + private final IntUnaryOperator sampleFunction; + + SampleReader( + RandomAccessVectorValues.Floats origin, int sampleSize, IntUnaryOperator sampleFunction) { + this.origin = origin; + this.sampleSize = sampleSize; + this.sampleFunction = sampleFunction; + } + + @Override + public int size() { + return sampleSize; + } + + @Override + public int dimension() { + return origin.dimension(); + } + + @Override + public Floats copy() throws IOException { + throw new IllegalStateException("Not supported"); + } + + @Override + public IndexInput getSlice() { + return origin.getSlice(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { + return origin.vectorValue(sampleFunction.applyAsInt(targetOrd)); + } + + @Override + public int getVectorByteLength() { + return origin.getVectorByteLength(); + } + + @Override + public int ordToDoc(int ord) { + throw new IllegalStateException("Not supported"); + } + + @Override + public Bits getAcceptOrds(Bits acceptDocs) { + throw new IllegalStateException("Not supported"); + } + + public static SampleReader createSampleReader( + RandomAccessVectorValues.Floats origin, int k, long seed) { + int[] samples = reservoirSample(origin.size(), k, seed); + return new SampleReader(origin, samples.length, i -> samples[i]); + } + + /** + * Sample k elements from n elements according to reservoir sampling algorithm. + * + * @param n number of elements + * @param k number of samples + * @param seed random seed + * @return array of k samples + */ + public static int[] reservoirSample(int n, int k, long seed) { + Random rnd = new Random(seed); + int[] reservoir = new int[k]; + for (int i = 0; i < k; i++) { + reservoir[i] = i; + } + for (int i = k; i < n; i++) { + int j = rnd.nextInt(i + 1); + if (j < k) { + reservoir[j] = i; + } + } + return reservoir; + } + + /** + * Sample k elements from the origin array using reservoir sampling algorithm. + * + * @param origin original array + * @param k number of samples + * @param seed random seed + * @return array of k samples + */ + public static int[] reservoirSampleFromArray(int[] origin, int k, long seed) { + Random rnd = new Random(seed); + if (k >= origin.length) { + return origin; + } + int[] reservoir = new int[k]; + for (int i = 0; i < k; i++) { + reservoir[i] = origin[i]; + } + for (int i = k; i < origin.length; i++) { + int j = rnd.nextInt(i + 1); + if (j < k) { + reservoir[j] = origin[i]; + } + } + return reservoir; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/package-info.java new file mode 100644 index 00000000000..477fa361de3 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/quantization/package-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ + +/** This package implements KMeans algorithm for clustering vectors */ +package org.apache.lucene.sandbox.codecs.quantization; diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/quantization/TestKMeans.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/quantization/TestKMeans.java new file mode 100644 index 00000000000..61c0e58c91e --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/codecs/quantization/TestKMeans.java @@ -0,0 +1,142 @@ +/* + * 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.sandbox.codecs.quantization; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; + +public class TestKMeans extends LuceneTestCase { + + public void testKMeansAPI() throws IOException { + int nClusters = random().nextInt(1, 10); + int nVectors = random().nextInt(nClusters * 100, nClusters * 200); + int dims = random().nextInt(2, 20); + int randIdx = random().nextInt(VectorSimilarityFunction.values().length); + VectorSimilarityFunction similarityFunction = VectorSimilarityFunction.values()[randIdx]; + RandomAccessVectorValues.Floats vectors = generateData(nVectors, dims, nClusters); + + // default case + { + KMeans.Results results = KMeans.cluster(vectors, similarityFunction, nClusters); + assertEquals(nClusters, results.centroids().length); + assertEquals(nVectors, results.vectorCentroids().length); + } + // expert case + { + boolean assignCentroidsToVectors = random().nextBoolean(); + int randIdx2 = random().nextInt(KMeans.KmeansInitializationMethod.values().length); + KMeans.KmeansInitializationMethod initializationMethod = + KMeans.KmeansInitializationMethod.values()[randIdx2]; + int restarts = random().nextInt(1, 6); + int iters = random().nextInt(1, 10); + int sampleSize = random().nextInt(10, nVectors * 2); + + KMeans.Results results = + KMeans.cluster( + vectors, + nClusters, + assignCentroidsToVectors, + random().nextLong(), + initializationMethod, + similarityFunction == VectorSimilarityFunction.COSINE, + restarts, + iters, + sampleSize); + assertEquals(nClusters, results.centroids().length); + if (assignCentroidsToVectors) { + assertEquals(nVectors, results.vectorCentroids().length); + } else { + assertNull(results.vectorCentroids()); + } + } + } + + public void testKMeansSpecialCases() throws IOException { + { + // nClusters > nVectors + int nClusters = 20; + int nVectors = 10; + RandomAccessVectorValues.Floats vectors = generateData(nVectors, 5, nClusters); + KMeans.Results results = + KMeans.cluster(vectors, VectorSimilarityFunction.EUCLIDEAN, nClusters); + // assert that we get 1 centroid, as nClusters will be adjusted + assertEquals(1, results.centroids().length); + assertEquals(nVectors, results.vectorCentroids().length); + } + { + // small sample size + int sampleSize = 2; + int nClusters = 2; + int nVectors = 300; + RandomAccessVectorValues.Floats vectors = generateData(nVectors, 5, nClusters); + KMeans.KmeansInitializationMethod initializationMethod = + KMeans.KmeansInitializationMethod.PLUS_PLUS; + KMeans.Results results = + KMeans.cluster( + vectors, + nClusters, + true, + random().nextLong(), + initializationMethod, + false, + 1, + 2, + sampleSize); + assertEquals(nClusters, results.centroids().length); + assertEquals(nVectors, results.vectorCentroids().length); + } + { + // test unassigned centroids + int nClusters = 4; + int nVectors = 400; + RandomAccessVectorValues.Floats vectors = generateData(nVectors, 5, nClusters); + KMeans.Results results = + KMeans.cluster(vectors, VectorSimilarityFunction.EUCLIDEAN, nClusters); + float[][] centroids = results.centroids(); + List unassignedIdxs = List.of(0, 3); + KMeans.assignCentroids(vectors, centroids, unassignedIdxs); + assertEquals(nClusters, centroids.length); + } + } + + private static RandomAccessVectorValues.Floats generateData( + int nSamples, int nDims, int nClusters) { + List vectors = new ArrayList<>(nSamples); + float[][] centroids = new float[nClusters][nDims]; + // Generate random centroids + for (int i = 0; i < nClusters; i++) { + for (int j = 0; j < nDims; j++) { + centroids[i][j] = random().nextFloat() * 100; + } + } + // Generate data points around centroids + for (int i = 0; i < nSamples; i++) { + int cluster = random().nextInt(nClusters); + float[] vector = new float[nDims]; + for (int j = 0; j < nDims; j++) { + vector[j] = centroids[cluster][j] + random().nextFloat() * 10 - 5; + } + vectors.add(vector); + } + return RandomAccessVectorValues.fromFloats(vectors, nDims); + } +} From 43c80117dd51a01e0585242a89de2126a3fea059 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 5 Aug 2024 12:29:14 -0400 Subject: [PATCH 49/66] Fix ScalarQuantization when used with COSINE similarity (#13615) When quantizing vectors in a COSINE vector space, we normalize them. However, there is a bug when building the quantizer quantiles and we didn't always use the normalized vectors. Consequently, we would end up with poorly configured quantiles and recall will drop significantly (especially in sensitive cases like int4). closes: #13614 --- lucene/CHANGES.txt | 3 + .../Lucene99ScalarQuantizedVectorsWriter.java | 100 ++++++++++++------ .../org/apache/lucene/util/VectorUtil.java | 9 +- .../util/quantization/ScalarQuantizer.java | 3 + ...estLucene99HnswQuantizedVectorsFormat.java | 21 ++-- ...tLucene99ScalarQuantizedVectorsFormat.java | 19 ++-- ...tLucene99ScalarQuantizedVectorsWriter.java | 9 +- .../quantization/TestScalarQuantizer.java | 18 +++- 8 files changed, 118 insertions(+), 64 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index fb91b89a2a4..739887b0b91 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -350,6 +350,9 @@ Bug Fixes * GITHUB#13553: Correct RamUsageEstimate for scalar quantized knn vector formats so that raw vectors are correctly accounted for. (Ben Trent) +* GITHUB#13615: Correct scalar quantization when used in conjunction with COSINE similarity. Vectors are normalized + before quantization to ensure the cosine similarity is correctly calculated. (Ben Trent) + Other -------------------- (No changes) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java index 311f2df435e..e477fec75e5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java @@ -677,6 +677,10 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite Float confidenceInterval, byte bits) throws IOException { + if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) { + floatVectorValues = new NormalizedFloatVectorValues(floatVectorValues); + vectorSimilarityFunction = VectorSimilarityFunction.DOT_PRODUCT; + } if (confidenceInterval != null && confidenceInterval == DYNAMIC_CONFIDENCE_INTERVAL) { return ScalarQuantizer.fromVectorsAutoInterval( floatVectorValues, vectorSimilarityFunction, numVectors, bits); @@ -797,10 +801,9 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite if (floatVectors.size() == 0) { return new ScalarQuantizer(0, 0, bits); } - FloatVectorValues floatVectorValues = new FloatVectorWrapper(floatVectors, normalize); ScalarQuantizer quantizer = buildScalarQuantizer( - floatVectorValues, + new FloatVectorWrapper(floatVectors), floatVectors.size(), fieldInfo.getVectorSimilarityFunction(), confidenceInterval, @@ -851,14 +854,10 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite static class FloatVectorWrapper extends FloatVectorValues { private final List vectorList; - private final float[] copy; - private final boolean normalize; protected int curDoc = -1; - FloatVectorWrapper(List vectorList, boolean normalize) { + FloatVectorWrapper(List vectorList) { this.vectorList = vectorList; - this.copy = new float[vectorList.get(0).length]; - this.normalize = normalize; } @Override @@ -876,11 +875,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite if (curDoc == -1 || curDoc >= vectorList.size()) { throw new IOException("Current doc not set or too many iterations"); } - if (normalize) { - System.arraycopy(vectorList.get(curDoc), 0, copy, 0, copy.length); - VectorUtil.l2normalize(copy); - return copy; - } return vectorList.get(curDoc); } @@ -949,13 +943,16 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite // quantization? || scalarQuantizer.getBits() <= 4 || shouldRequantize(reader.getQuantizationState(fieldInfo.name), scalarQuantizer)) { + FloatVectorValues toQuantize = + mergeState.knnVectorsReaders[i].getFloatVectorValues(fieldInfo.name); + if (fieldInfo.getVectorSimilarityFunction() == VectorSimilarityFunction.COSINE) { + toQuantize = new NormalizedFloatVectorValues(toQuantize); + } sub = new QuantizedByteVectorValueSub( mergeState.docMaps[i], new QuantizedFloatVectorValues( - mergeState.knnVectorsReaders[i].getFloatVectorValues(fieldInfo.name), - fieldInfo.getVectorSimilarityFunction(), - scalarQuantizer)); + toQuantize, fieldInfo.getVectorSimilarityFunction(), scalarQuantizer)); } else { sub = new QuantizedByteVectorValueSub( @@ -1042,7 +1039,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite private final FloatVectorValues values; private final ScalarQuantizer quantizer; private final byte[] quantizedVector; - private final float[] normalizedVector; private float offsetValue = 0f; private final VectorSimilarityFunction vectorSimilarityFunction; @@ -1055,11 +1051,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite this.quantizer = quantizer; this.quantizedVector = new byte[values.dimension()]; this.vectorSimilarityFunction = vectorSimilarityFunction; - if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) { - this.normalizedVector = new float[values.dimension()]; - } else { - this.normalizedVector = null; - } } @Override @@ -1111,15 +1102,8 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite } private void quantize() throws IOException { - if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) { - System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length); - VectorUtil.l2normalize(normalizedVector); - offsetValue = - quantizer.quantize(normalizedVector, quantizedVector, vectorSimilarityFunction); - } else { - offsetValue = - quantizer.quantize(values.vectorValue(), quantizedVector, vectorSimilarityFunction); - } + offsetValue = + quantizer.quantize(values.vectorValue(), quantizedVector, vectorSimilarityFunction); } } @@ -1216,4 +1200,60 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite throw new UnsupportedOperationException(); } } + + static final class NormalizedFloatVectorValues extends FloatVectorValues { + private final FloatVectorValues values; + private final float[] normalizedVector; + int curDoc = -1; + + public NormalizedFloatVectorValues(FloatVectorValues values) { + this.values = values; + this.normalizedVector = new float[values.dimension()]; + } + + @Override + public int dimension() { + return values.dimension(); + } + + @Override + public int size() { + return values.size(); + } + + @Override + public float[] vectorValue() throws IOException { + return normalizedVector; + } + + @Override + public VectorScorer scorer(float[] query) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int docID() { + return values.docID(); + } + + @Override + public int nextDoc() throws IOException { + curDoc = values.nextDoc(); + if (curDoc != NO_MORE_DOCS) { + System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length); + VectorUtil.l2normalize(normalizedVector); + } + return curDoc; + } + + @Override + public int advance(int target) throws IOException { + curDoc = values.advance(target); + if (curDoc != NO_MORE_DOCS) { + System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length); + VectorUtil.l2normalize(normalizedVector); + } + return curDoc; + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index f122ae95544..e1c3978cff3 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java @@ -47,6 +47,8 @@ import org.apache.lucene.internal.vectorization.VectorizationProvider; */ public final class VectorUtil { + private static final float EPSILON = 1e-4f; + private static final VectorUtilSupport IMPL = VectorizationProvider.getInstance().getVectorUtilSupport(); @@ -121,6 +123,11 @@ public final class VectorUtil { return v; } + public static boolean isUnitVector(float[] v) { + double l1norm = IMPL.dotProduct(v, v); + return Math.abs(l1norm - 1.0d) <= EPSILON; + } + /** * Modifies the argument to be unit length, dividing by its l2-norm. * @@ -138,7 +145,7 @@ public final class VectorUtil { return v; } } - if (Math.abs(l1norm - 1.0d) <= 1e-5) { + if (Math.abs(l1norm - 1.0d) <= EPSILON) { return v; } int dim = v.length; diff --git a/lucene/core/src/java/org/apache/lucene/util/quantization/ScalarQuantizer.java b/lucene/core/src/java/org/apache/lucene/util/quantization/ScalarQuantizer.java index fb07e005571..44c0ac5aca4 100644 --- a/lucene/core/src/java/org/apache/lucene/util/quantization/ScalarQuantizer.java +++ b/lucene/core/src/java/org/apache/lucene/util/quantization/ScalarQuantizer.java @@ -30,6 +30,7 @@ import org.apache.lucene.search.HitQueue; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.util.IntroSelector; import org.apache.lucene.util.Selector; +import org.apache.lucene.util.VectorUtil; /** * Will scalar quantize float vectors into `int8` byte values. This is a lossy transformation. @@ -113,6 +114,7 @@ public class ScalarQuantizer { */ public float quantize(float[] src, byte[] dest, VectorSimilarityFunction similarityFunction) { assert src.length == dest.length; + assert similarityFunction != VectorSimilarityFunction.COSINE || VectorUtil.isUnitVector(src); float correction = 0; for (int i = 0; i < src.length; i++) { correction += quantizeFloat(src[i], dest, i); @@ -332,6 +334,7 @@ public class ScalarQuantizer { int totalVectorCount, byte bits) throws IOException { + assert function != VectorSimilarityFunction.COSINE; if (totalVectorCount == 0) { return new ScalarQuantizer(0f, 0f, bits); } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java index 8e69e833b98..3098ca7fbf3 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java @@ -35,6 +35,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; import org.apache.lucene.index.CodecReader; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; @@ -127,7 +128,6 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat // create lucene directory with codec int numVectors = 1 + random().nextInt(50); VectorSimilarityFunction similarityFunction = randomSimilarity(); - boolean normalize = similarityFunction == VectorSimilarityFunction.COSINE; int dim = random().nextInt(64) + 1; if (dim % 2 == 1) { dim++; @@ -136,25 +136,16 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat for (int i = 0; i < numVectors; i++) { vectors.add(randomVector(dim)); } + FloatVectorValues toQuantize = + new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors); ScalarQuantizer scalarQuantizer = - confidenceInterval != null && confidenceInterval == 0f - ? ScalarQuantizer.fromVectorsAutoInterval( - new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors, normalize), - similarityFunction, - numVectors, - (byte) bits) - : ScalarQuantizer.fromVectors( - new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors, normalize), - confidenceInterval == null - ? Lucene99ScalarQuantizedVectorsFormat.calculateDefaultConfidenceInterval(dim) - : confidenceInterval, - numVectors, - (byte) bits); + Lucene99ScalarQuantizedVectorsWriter.buildScalarQuantizer( + toQuantize, numVectors, similarityFunction, confidenceInterval, (byte) bits); float[] expectedCorrections = new float[numVectors]; byte[][] expectedVectors = new byte[numVectors][]; for (int i = 0; i < numVectors; i++) { float[] vector = vectors.get(i); - if (normalize) { + if (similarityFunction == VectorSimilarityFunction.COSINE) { float[] copy = new float[vector.length]; System.arraycopy(vector, 0, copy, 0, copy.length); VectorUtil.l2normalize(copy); diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java index b221cb19dde..094d90ba5a2 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java @@ -114,19 +114,12 @@ public class TestLucene99ScalarQuantizedVectorsFormat extends BaseKnnVectorsForm vectors.add(randomVector(dim)); } ScalarQuantizer scalarQuantizer = - confidenceInterval != null && confidenceInterval == 0f - ? ScalarQuantizer.fromVectorsAutoInterval( - new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors, normalize), - similarityFunction, - numVectors, - (byte) bits) - : ScalarQuantizer.fromVectors( - new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors, normalize), - confidenceInterval == null - ? Lucene99ScalarQuantizedVectorsFormat.calculateDefaultConfidenceInterval(dim) - : confidenceInterval, - numVectors, - (byte) bits); + Lucene99ScalarQuantizedVectorsWriter.buildScalarQuantizer( + new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors), + numVectors, + similarityFunction, + confidenceInterval, + (byte) bits); float[] expectedCorrections = new float[numVectors]; byte[][] expectedVectors = new byte[numVectors][]; for (int i = 0; i < numVectors; i++) { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsWriter.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsWriter.java index 34af5ea3e15..0bf2a4ef6b8 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsWriter.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsWriter.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.VectorUtil; import org.apache.lucene.util.quantization.ScalarQuantizer; public class TestLucene99ScalarQuantizedVectorsWriter extends LuceneTestCase { @@ -87,12 +88,12 @@ public class TestLucene99ScalarQuantizedVectorsWriter extends LuceneTestCase { for (int i = 0; i < 30; i++) { float[] vector = new float[] {i, i + 1, i + 2, i + 3}; vectors.add(vector); + if (vectorSimilarityFunction == VectorSimilarityFunction.DOT_PRODUCT) { + VectorUtil.l2normalize(vector); + } } FloatVectorValues vectorValues = - new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper( - vectors, - vectorSimilarityFunction == VectorSimilarityFunction.COSINE - || vectorSimilarityFunction == VectorSimilarityFunction.DOT_PRODUCT); + new Lucene99ScalarQuantizedVectorsWriter.FloatVectorWrapper(vectors); ScalarQuantizer scalarQuantizer = Lucene99ScalarQuantizedVectorsWriter.buildScalarQuantizer( vectorValues, 30, vectorSimilarityFunction, confidenceInterval, bits); diff --git a/lucene/core/src/test/org/apache/lucene/util/quantization/TestScalarQuantizer.java b/lucene/core/src/test/org/apache/lucene/util/quantization/TestScalarQuantizer.java index 0ee2c01aa29..48eb7ce651c 100644 --- a/lucene/core/src/test/org/apache/lucene/util/quantization/TestScalarQuantizer.java +++ b/lucene/core/src/test/org/apache/lucene/util/quantization/TestScalarQuantizer.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.search.VectorScorer; import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.VectorUtil; public class TestScalarQuantizer extends LuceneTestCase { @@ -33,8 +34,16 @@ public class TestScalarQuantizer extends LuceneTestCase { int dims = random().nextInt(9) + 1; int numVecs = random().nextInt(9) + 10; float[][] floats = randomFloats(numVecs, dims); + if (function == VectorSimilarityFunction.COSINE) { + for (float[] v : floats) { + VectorUtil.l2normalize(v); + } + } for (byte bits : new byte[] {4, 7}) { FloatVectorValues floatVectorValues = fromFloats(floats); + if (function == VectorSimilarityFunction.COSINE) { + function = VectorSimilarityFunction.DOT_PRODUCT; + } ScalarQuantizer scalarQuantizer = random().nextBoolean() ? ScalarQuantizer.fromVectors(floatVectorValues, 0.9f, numVecs, bits) @@ -63,11 +72,15 @@ public class TestScalarQuantizer extends LuceneTestCase { expectThrows( IllegalStateException.class, () -> ScalarQuantizer.fromVectors(floatVectorValues, 0.9f, numVecs, bits)); + VectorSimilarityFunction actualFunction = + function == VectorSimilarityFunction.COSINE + ? VectorSimilarityFunction.DOT_PRODUCT + : function; expectThrows( IllegalStateException.class, () -> ScalarQuantizer.fromVectorsAutoInterval( - floatVectorValues, function, numVecs, bits)); + floatVectorValues, actualFunction, numVecs, bits)); } } } @@ -185,6 +198,9 @@ public class TestScalarQuantizer extends LuceneTestCase { VectorSimilarityFunction similarityFunction = VectorSimilarityFunction.DOT_PRODUCT; float[][] floats = randomFloats(numVecs, dims); + for (float[] v : floats) { + VectorUtil.l2normalize(v); + } FloatVectorValues floatVectorValues = fromFloats(floats); ScalarQuantizer scalarQuantizer = ScalarQuantizer.fromVectorsAutoInterval( From e0e5d81df802fba5570d40d44aec725c8dadfbaa Mon Sep 17 00:00:00 2001 From: Kaival Parikh <46070017+kaivalnp@users.noreply.github.com> Date: Tue, 6 Aug 2024 05:42:19 +0530 Subject: [PATCH 50/66] Add timeout support to AbstractVectorSimilarityQuery (#13285) Co-authored-by: Kaival Parikh --- lucene/CHANGES.txt | 3 + .../search/AbstractVectorSimilarityQuery.java | 120 ++++++++------ .../search/ByteVectorSimilarityQuery.java | 10 +- .../search/FloatVectorSimilarityQuery.java | 10 +- .../BaseVectorSimilarityQueryTestCase.java | 150 ++++++++++++++++++ 5 files changed, 237 insertions(+), 56 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 739887b0b91..265c339145e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -288,6 +288,9 @@ Improvements * GITHUB#13625: Remove BitSet#nextSetBit code duplication. (Greg Miller) +* GITHUB#13285: Early terminate graph searches of AbstractVectorSimilarityQuery to follow timeout set from + IndexSearcher#setTimeout(QueryTimeout). (Kaival Parikh) + Optimizations --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java b/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java index 77a5ff6f24f..75d639c08fe 100644 --- a/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Objects; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.QueryTimeout; +import org.apache.lucene.search.knn.KnnCollectorManager; import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSetIterator; import org.apache.lucene.util.Bits; @@ -58,10 +60,19 @@ abstract class AbstractVectorSimilarityQuery extends Query { this.filter = filter; } + protected KnnCollectorManager getKnnCollectorManager() { + return (visitedLimit, context) -> + new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitedLimit); + } + abstract VectorScorer createVectorScorer(LeafReaderContext context) throws IOException; protected abstract TopDocs approximateSearch( - LeafReaderContext context, Bits acceptDocs, int visitLimit) throws IOException; + LeafReaderContext context, + Bits acceptDocs, + int visitLimit, + KnnCollectorManager knnCollectorManager) + throws IOException; @Override public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) @@ -72,6 +83,10 @@ abstract class AbstractVectorSimilarityQuery extends Query { ? null : searcher.createWeight(searcher.rewrite(filter), ScoreMode.COMPLETE_NO_SCORES, 1); + final QueryTimeout queryTimeout = searcher.getTimeout(); + final TimeLimitingKnnCollectorManager timeLimitingKnnCollectorManager = + new TimeLimitingKnnCollectorManager(getKnnCollectorManager(), queryTimeout); + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { if (filterWeight != null) { @@ -103,16 +118,14 @@ abstract class AbstractVectorSimilarityQuery extends Query { public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { LeafReader leafReader = context.reader(); Bits liveDocs = leafReader.getLiveDocs(); - final Scorer vectorSimilarityScorer; + // If there is no filter if (filterWeight == null) { // Return exhaustive results - TopDocs results = approximateSearch(context, liveDocs, Integer.MAX_VALUE); - if (results.scoreDocs.length == 0) { - return null; - } - vectorSimilarityScorer = - VectorSimilarityScorer.fromScoreDocs(this, boost, results.scoreDocs); + TopDocs results = + approximateSearch( + context, liveDocs, Integer.MAX_VALUE, timeLimitingKnnCollectorManager); + return VectorSimilarityScorerSupplier.fromScoreDocs(boost, results.scoreDocs); } else { Scorer scorer = filterWeight.scorer(context); if (scorer == null) { @@ -143,27 +156,23 @@ abstract class AbstractVectorSimilarityQuery extends Query { } // Perform an approximate search - TopDocs results = approximateSearch(context, acceptDocs, cardinality); + TopDocs results = + approximateSearch(context, acceptDocs, cardinality, timeLimitingKnnCollectorManager); - // If the limit was exhausted - if (results.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) { - // Return a lazy-loading iterator - vectorSimilarityScorer = - VectorSimilarityScorer.fromAcceptDocs( - this, - boost, - createVectorScorer(context), - new BitSetIterator(acceptDocs, cardinality), - resultSimilarity); - } else if (results.scoreDocs.length == 0) { - return null; - } else { + if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO + // Return partial results only when timeout is met + || (queryTimeout != null && queryTimeout.shouldExit())) { // Return an iterator over the collected results - vectorSimilarityScorer = - VectorSimilarityScorer.fromScoreDocs(this, boost, results.scoreDocs); + return VectorSimilarityScorerSupplier.fromScoreDocs(boost, results.scoreDocs); + } else { + // Return a lazy-loading iterator + return VectorSimilarityScorerSupplier.fromAcceptDocs( + boost, + createVectorScorer(context), + new BitSetIterator(acceptDocs, cardinality), + resultSimilarity); } } - return new DefaultScorerSupplier(vectorSimilarityScorer); } @Override @@ -197,16 +206,20 @@ abstract class AbstractVectorSimilarityQuery extends Query { return Objects.hash(field, traversalSimilarity, resultSimilarity, filter); } - private static class VectorSimilarityScorer extends Scorer { + private static class VectorSimilarityScorerSupplier extends ScorerSupplier { final DocIdSetIterator iterator; final float[] cachedScore; - VectorSimilarityScorer(DocIdSetIterator iterator, float[] cachedScore) { + VectorSimilarityScorerSupplier(DocIdSetIterator iterator, float[] cachedScore) { this.iterator = iterator; this.cachedScore = cachedScore; } - static VectorSimilarityScorer fromScoreDocs(Weight weight, float boost, ScoreDoc[] scoreDocs) { + static VectorSimilarityScorerSupplier fromScoreDocs(float boost, ScoreDoc[] scoreDocs) { + if (scoreDocs.length == 0) { + return null; + } + // Sort in ascending order of docid Arrays.sort(scoreDocs, Comparator.comparingInt(scoreDoc -> scoreDoc.doc)); @@ -252,18 +265,15 @@ abstract class AbstractVectorSimilarityQuery extends Query { } }; - return new VectorSimilarityScorer(iterator, cachedScore); + return new VectorSimilarityScorerSupplier(iterator, cachedScore); } - static VectorSimilarityScorer fromAcceptDocs( - Weight weight, - float boost, - VectorScorer scorer, - DocIdSetIterator acceptDocs, - float threshold) { + static VectorSimilarityScorerSupplier fromAcceptDocs( + float boost, VectorScorer scorer, DocIdSetIterator acceptDocs, float threshold) { if (scorer == null) { return null; } + float[] cachedScore = new float[1]; DocIdSetIterator vectorIterator = scorer.iterator(); DocIdSetIterator conjunction = @@ -281,27 +291,37 @@ abstract class AbstractVectorSimilarityQuery extends Query { } }; - return new VectorSimilarityScorer(iterator, cachedScore); + return new VectorSimilarityScorerSupplier(iterator, cachedScore); } @Override - public int docID() { - return iterator.docID(); + public Scorer get(long leadCost) { + return new Scorer() { + @Override + public int docID() { + return iterator.docID(); + } + + @Override + public DocIdSetIterator iterator() { + return iterator; + } + + @Override + public float getMaxScore(int upTo) { + return Float.POSITIVE_INFINITY; + } + + @Override + public float score() { + return cachedScore[0]; + } + }; } @Override - public DocIdSetIterator iterator() { - return iterator; - } - - @Override - public float getMaxScore(int upTo) { - return Float.POSITIVE_INFINITY; - } - - @Override - public float score() { - return cachedScore[0]; + public long cost() { + return iterator.cost(); } } } diff --git a/lucene/core/src/java/org/apache/lucene/search/ByteVectorSimilarityQuery.java b/lucene/core/src/java/org/apache/lucene/search/ByteVectorSimilarityQuery.java index bd2190121ab..c547f1face7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ByteVectorSimilarityQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ByteVectorSimilarityQuery.java @@ -23,6 +23,7 @@ import java.util.Objects; import org.apache.lucene.document.KnnByteVectorField; import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.knn.KnnCollectorManager; import org.apache.lucene.util.Bits; /** @@ -106,10 +107,13 @@ public class ByteVectorSimilarityQuery extends AbstractVectorSimilarityQuery { @Override @SuppressWarnings("resource") - protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitLimit) + protected TopDocs approximateSearch( + LeafReaderContext context, + Bits acceptDocs, + int visitLimit, + KnnCollectorManager knnCollectorManager) throws IOException { - KnnCollector collector = - new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitLimit); + KnnCollector collector = knnCollectorManager.newCollector(visitLimit, context); context.reader().searchNearestVectors(field, target, collector, acceptDocs); return collector.topDocs(); } diff --git a/lucene/core/src/java/org/apache/lucene/search/FloatVectorSimilarityQuery.java b/lucene/core/src/java/org/apache/lucene/search/FloatVectorSimilarityQuery.java index 3dc92482a77..4c7078ac140 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FloatVectorSimilarityQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/FloatVectorSimilarityQuery.java @@ -23,6 +23,7 @@ import java.util.Objects; import org.apache.lucene.document.KnnFloatVectorField; import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.knn.KnnCollectorManager; import org.apache.lucene.util.Bits; import org.apache.lucene.util.VectorUtil; @@ -108,10 +109,13 @@ public class FloatVectorSimilarityQuery extends AbstractVectorSimilarityQuery { @Override @SuppressWarnings("resource") - protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitLimit) + protected TopDocs approximateSearch( + LeafReaderContext context, + Bits acceptDocs, + int visitLimit, + KnnCollectorManager knnCollectorManager) throws IOException { - KnnCollector collector = - new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitLimit); + KnnCollector collector = knnCollectorManager.newCollector(visitLimit, context); context.reader().searchNearestVectors(field, target, collector, acceptDocs); return collector.topDocs(); } diff --git a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java index 3347b9478dd..1e32c07b665 100644 --- a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java +++ b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java @@ -16,6 +16,8 @@ */ package org.apache.lucene.search; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + import java.io.IOException; import java.util.Arrays; import java.util.HashMap; @@ -32,6 +34,8 @@ import org.apache.lucene.document.IntField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.QueryTimeout; import org.apache.lucene.index.Term; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; @@ -475,6 +479,62 @@ abstract class BaseVectorSimilarityQueryTestCase< } } + /** Test that the query times out correctly. */ + public void testTimeout() throws IOException { + V[] vectors = getRandomVectors(numDocs, dim); + V queryVector = getRandomVector(dim); + + try (Directory indexStore = getIndexStore(vectors); + IndexReader reader = DirectoryReader.open(indexStore)) { + IndexSearcher searcher = newSearcher(reader); + + // This query is cacheable, explicitly prevent it + searcher.setQueryCache(null); + + Query query = + new CountingQuery( + getVectorQuery( + vectorField, + queryVector, + Float.NEGATIVE_INFINITY, + Float.NEGATIVE_INFINITY, + null)); + + assertEquals(numDocs, searcher.count(query)); // Expect some results without timeout + + searcher.setTimeout(() -> true); // Immediately timeout + assertEquals(0, searcher.count(query)); // Expect no results with the timeout + + searcher.setTimeout(new CountingQueryTimeout(numDocs - 1)); // Do not score all docs + int count = searcher.count(query); + assertTrue( + "0 < count=" + count + " < numDocs=" + numDocs, + count > 0 && count < numDocs); // Expect partial results + + // Test timeout with filter + int numFiltered = random().nextInt(numDocs / 2, numDocs); + Query filter = IntField.newSetQuery(idField, getFiltered(numFiltered)); + Query filteredQuery = + new CountingQuery( + getVectorQuery( + vectorField, + queryVector, + Float.NEGATIVE_INFINITY, + Float.NEGATIVE_INFINITY, + filter)); + + searcher.setTimeout(() -> false); // Set a timeout which is never met + assertEquals(numFiltered, searcher.count(filteredQuery)); + + searcher.setTimeout( + new CountingQueryTimeout(numFiltered - 1)); // Timeout before scoring all filtered docs + int filteredCount = searcher.count(filteredQuery); + assertTrue( + "0 < filteredCount=" + filteredCount + " < numFiltered=" + numFiltered, + filteredCount > 0 && filteredCount < numFiltered); // Expect partial results + } + } + private float getSimilarity(V[] vectors, V queryVector, int targetVisited) { assertTrue(targetVisited >= 0 && targetVisited <= numDocs); if (targetVisited == 0) { @@ -526,4 +586,94 @@ abstract class BaseVectorSimilarityQueryTestCase< } return dir; } + + private static class CountingQueryTimeout implements QueryTimeout { + private int remaining; + + public CountingQueryTimeout(int count) { + remaining = count; + } + + @Override + public boolean shouldExit() { + if (remaining > 0) { + remaining--; + return false; + } + return true; + } + } + + /** + * A {@link Query} that emulates {@link Weight#count(LeafReaderContext)} by counting number of + * docs of underlying {@link Scorer#iterator()}. TODO: This is a workaround to count partial + * results of {@link #delegate} because {@link TimeLimitingBulkScorer} immediately discards + * results after timeout. + */ + private static class CountingQuery extends Query { + private final Query delegate; + + private CountingQuery(Query delegate) { + this.delegate = delegate; + } + + @Override + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) + throws IOException { + return new Weight(this) { + final Weight delegateWeight = delegate.createWeight(searcher, scoreMode, boost); + + @Override + public Explanation explain(LeafReaderContext context, int doc) throws IOException { + return delegateWeight.explain(context, doc); + } + + @Override + public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { + return delegateWeight.scorerSupplier(context); + } + + @Override + public int count(LeafReaderContext context) throws IOException { + Scorer scorer = scorer(context); + if (scorer == null) { + return 0; + } + + int count = 0; + DocIdSetIterator iterator = scorer.iterator(); + while (iterator.nextDoc() != NO_MORE_DOCS) { + count++; + } + return count; + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return delegateWeight.isCacheable(ctx); + } + }; + } + + @Override + public String toString(String field) { + return String.format( + Locale.ROOT, "%s[%s]", getClass().getSimpleName(), delegate.toString(field)); + } + + @Override + public void visit(QueryVisitor visitor) { + visitor.visitLeaf(this); + } + + @Override + public boolean equals(Object obj) { + return sameClassAs(obj) && delegate.equals(((CountingQuery) obj).delegate); + } + + @Override + public int hashCode() { + return delegate.hashCode(); + } + } } From 82210189ee849dfe5f06f2c8195cfaf05f43614c Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 6 Aug 2024 14:30:03 +0200 Subject: [PATCH 51/66] Add missing test for Lucene912PostingsFormat. --- .../codecs/lucene912/TestForDeltaUtil.java | 89 +++++++++++++++ .../lucene/codecs/lucene912/TestForUtil.java | 94 ++++++++++++++++ .../lucene/codecs/lucene912/TestPForUtil.java | 104 ++++++++++++++++++ .../codecs/lucene912/TestPostingsUtil.java | 49 +++++++++ 4 files changed, 336 insertions(+) create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java new file mode 100644 index 00000000000..3c201ce6835 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java @@ -0,0 +1,89 @@ +/* + * 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.codecs.lucene912; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestForDeltaUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 1, 31 - 7); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 1, (int) PackedInts.maxValue(bpv)); + } + } + + final Directory d = new ByteBuffersDirectory(); + final long endPointer; + + { + // encode + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(new ForUtil()); + + for (int i = 0; i < iterations; ++i) { + long[] source = new long[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + } + forDeltaUtil.encodeDeltas(source, out); + } + endPointer = out.getFilePointer(); + out.close(); + } + + { + // decode + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(new ForUtil()); + for (int i = 0; i < iterations; ++i) { + long base = 0; + final long[] restored = new long[ForUtil.BLOCK_SIZE]; + forDeltaUtil.decodeAndPrefixSum(in, base, restored); + final long[] expected = new long[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + expected[j] = values[i * ForUtil.BLOCK_SIZE + j]; + if (j > 0) { + expected[j] += expected[j - 1]; + } else { + expected[j] += base; + } + } + assertArrayEquals(Arrays.toString(restored), expected, restored); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + } + + d.close(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java new file mode 100644 index 00000000000..114a9d0415c --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java @@ -0,0 +1,94 @@ +/* + * 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.codecs.lucene912; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestForUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 1, 31); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv)); + } + } + + final Directory d = new ByteBuffersDirectory(); + final long endPointer; + + { + // encode + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final ForUtil forUtil = new ForUtil(); + + for (int i = 0; i < iterations; ++i) { + long[] source = new long[ForUtil.BLOCK_SIZE]; + long or = 0; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + or |= source[j]; + } + final int bpv = PackedInts.bitsRequired(or); + out.writeByte((byte) bpv); + forUtil.encode(source, bpv, out); + } + endPointer = out.getFilePointer(); + out.close(); + } + + { + // decode + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + final ForUtil forUtil = new ForUtil(); + for (int i = 0; i < iterations; ++i) { + final int bitsPerValue = in.readByte(); + final long currentFilePointer = in.getFilePointer(); + final long[] restored = new long[ForUtil.BLOCK_SIZE]; + forUtil.decode(bitsPerValue, in, restored); + int[] ints = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + ints[j] = Math.toIntExact(restored[j]); + } + assertArrayEquals( + Arrays.toString(ints), + ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE), + ints); + assertEquals(forUtil.numBytes(bitsPerValue), in.getFilePointer() - currentFilePointer); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + } + + d.close(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java new file mode 100644 index 00000000000..08fec7a3a33 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java @@ -0,0 +1,104 @@ +/* + * 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.codecs.lucene912; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestPForUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = createTestData(iterations, 31); + + final Directory d = new ByteBuffersDirectory(); + final long endPointer = encodeTestData(iterations, values, d); + + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + final PForUtil pforUtil = new PForUtil(new ForUtil()); + for (int i = 0; i < iterations; ++i) { + if (random().nextInt(5) == 0) { + pforUtil.skip(in); + continue; + } + final long[] restored = new long[ForUtil.BLOCK_SIZE]; + pforUtil.decode(in, restored); + int[] ints = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + ints[j] = Math.toIntExact(restored[j]); + } + assertArrayEquals( + Arrays.toString(ints), + ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE), + ints); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + + d.close(); + } + + private int[] createTestData(int iterations, int maxBpv) { + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 0, maxBpv); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv)); + if (random().nextInt(100) == 0) { + final int exceptionBpv; + if (random().nextInt(10) == 0) { + exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 9, 16), maxBpv); + } else { + exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 1, 8), maxBpv); + } + values[i * ForUtil.BLOCK_SIZE + j] |= random().nextInt(1 << (exceptionBpv - bpv)) << bpv; + } + } + } + + return values; + } + + private long encodeTestData(int iterations, int[] values, Directory d) throws IOException { + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final PForUtil pforUtil = new PForUtil(new ForUtil()); + + for (int i = 0; i < iterations; ++i) { + long[] source = new long[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + } + pforUtil.encode(source, out); + } + final long endPointer = out.getFilePointer(); + out.close(); + + return endPointer; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java new file mode 100644 index 00000000000..870f6e94e98 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java @@ -0,0 +1,49 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; + +public class TestPostingsUtil extends LuceneTestCase { + + // checks for bug described in https://github.com/apache/lucene/issues/13373 + public void testIntegerOverflow() throws IOException { + final int size = random().nextInt(1, ForUtil.BLOCK_SIZE); + final long[] docDeltaBuffer = new long[size]; + final long[] freqBuffer = new long[size]; + + final int delta = 1 << 30; + docDeltaBuffer[0] = delta; + try (Directory dir = newDirectory()) { + try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) { + // In old implementation, this would cause integer overflow exception. + PostingsUtil.writeVIntBlock(out, docDeltaBuffer, freqBuffer, size, true); + } + long[] restoredDocs = new long[size]; + long[] restoredFreqs = new long[size]; + try (IndexInput in = dir.openInput("test", IOContext.DEFAULT)) { + PostingsUtil.readVIntBlock(in, restoredDocs, restoredFreqs, size, true, true); + } + assertEquals(delta, restoredDocs[0]); + } + } +} From 9e831ee809729054fb01d80e4f5dc736e91e732a Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 7 Aug 2024 08:38:41 -0400 Subject: [PATCH 52/66] Add float|byte vector support to memory index (#13633) * Add float|byte vector support to memory index * adding changes --- lucene/CHANGES.txt | 2 + .../lucene/index/memory/MemoryIndex.java | 238 +++++++++++++++++- .../lucene/index/memory/TestMemoryIndex.java | 170 +++++++++++++ 3 files changed, 408 insertions(+), 2 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 265c339145e..f9b989d723f 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -290,6 +290,8 @@ Improvements * GITHUB#13285: Early terminate graph searches of AbstractVectorSimilarityQuery to follow timeout set from IndexSearcher#setTimeout(QueryTimeout). (Kaival Parikh) + +* GITHUB#13633: Add ability to read/write knn vector values to a MemoryIndex. (Ben Trent) Optimizations --------------------- diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index 08fb1cf6b5b..9dad312d431 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -36,15 +36,19 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; import org.apache.lucene.document.Document; import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.KnnByteVectorField; +import org.apache.lucene.document.KnnFloatVectorField; import org.apache.lucene.index.*; import org.apache.lucene.search.Collector; import org.apache.lucene.search.CollectorManager; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorable; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.SimpleCollector; +import org.apache.lucene.search.VectorScorer; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.Directory; import org.apache.lucene.util.ArrayUtil; @@ -636,6 +640,10 @@ public class MemoryIndex { if (field.fieldType().stored()) { storeValues(info, field); } + + if (field.fieldType().vectorDimension() > 0) { + storeVectorValues(info, field); + } } /** @@ -749,6 +757,56 @@ public class MemoryIndex { info.pointValues[info.pointValuesCount++] = BytesRef.deepCopyOf(pointValue); } + private void storeVectorValues(Info info, IndexableField vectorField) { + assert vectorField instanceof KnnFloatVectorField || vectorField instanceof KnnByteVectorField; + switch (info.fieldInfo.getVectorEncoding()) { + case BYTE -> { + if (vectorField instanceof KnnByteVectorField byteVectorField) { + if (info.byteVectorCount == 1) { + throw new IllegalArgumentException( + "Only one value per field allowed for byte vector field [" + + vectorField.name() + + "]"); + } + info.byteVectorCount++; + if (info.byteVectorValues == null) { + info.byteVectorValues = new byte[1][]; + } + info.byteVectorValues[0] = + ArrayUtil.copyOfSubArray( + byteVectorField.vectorValue(), 0, info.fieldInfo.getVectorDimension()); + return; + } + throw new IllegalArgumentException( + "Field [" + + vectorField.name() + + "] is not a byte vector field, but the field info is configured for byte vectors"); + } + case FLOAT32 -> { + if (vectorField instanceof KnnFloatVectorField floatVectorField) { + if (info.floatVectorCount == 1) { + throw new IllegalArgumentException( + "Only one value per field allowed for float vector field [" + + vectorField.name() + + "]"); + } + info.floatVectorCount++; + if (info.floatVectorValues == null) { + info.floatVectorValues = new float[1][]; + } + info.floatVectorValues[0] = + ArrayUtil.copyOfSubArray( + floatVectorField.vectorValue(), 0, info.fieldInfo.getVectorDimension()); + return; + } + throw new IllegalArgumentException( + "Field [" + + vectorField.name() + + "] is not a float vector field, but the field info is configured for float vectors"); + } + } + } + private void storeValues(Info info, IndexableField field) { if (info.storedValues == null) { info.storedValues = new ArrayList<>(); @@ -1148,6 +1206,18 @@ public class MemoryIndex { private BytesRef[] pointValues; + /** Number of float vectors added for this field */ + private int floatVectorCount; + + /** the float vectors added for this field */ + private float[][] floatVectorValues; + + /** Number of byte vectors added for this field */ + private int byteVectorCount; + + /** the byte vectors added for this field */ + private byte[][] byteVectorValues; + private byte[] minPackedValue; private byte[] maxPackedValue; @@ -1641,12 +1711,20 @@ public class MemoryIndex { @Override public FloatVectorValues getFloatVectorValues(String fieldName) { - return null; + Info info = fields.get(fieldName); + if (info == null || info.floatVectorValues == null) { + return null; + } + return new MemoryFloatVectorValues(info); } @Override public ByteVectorValues getByteVectorValues(String fieldName) { - return null; + Info info = fields.get(fieldName); + if (info == null || info.byteVectorValues == null) { + return null; + } + return new MemoryByteVectorValues(info); } @Override @@ -2204,4 +2282,160 @@ public class MemoryIndex { return super.clear(); } } + + private static final class MemoryFloatVectorValues extends FloatVectorValues { + private final Info info; + private int currentDoc = -1; + + MemoryFloatVectorValues(Info info) { + this.info = info; + } + + @Override + public int dimension() { + return info.fieldInfo.getVectorDimension(); + } + + @Override + public int size() { + return info.floatVectorCount; + } + + @Override + public float[] vectorValue() { + if (currentDoc == 0) { + return info.floatVectorValues[0]; + } else { + return null; + } + } + + @Override + public VectorScorer scorer(float[] query) { + if (query.length != info.fieldInfo.getVectorDimension()) { + throw new IllegalArgumentException( + "query vector dimension " + + query.length + + " does not match field dimension " + + info.fieldInfo.getVectorDimension()); + } + MemoryFloatVectorValues vectorValues = new MemoryFloatVectorValues(info); + return new VectorScorer() { + @Override + public float score() throws IOException { + return info.fieldInfo + .getVectorSimilarityFunction() + .compare(vectorValues.vectorValue(), query); + } + + @Override + public DocIdSetIterator iterator() { + return vectorValues; + } + }; + } + + @Override + public int docID() { + return currentDoc; + } + + @Override + public int nextDoc() { + int doc = ++currentDoc; + if (doc == 0) { + return doc; + } else { + return NO_MORE_DOCS; + } + } + + @Override + public int advance(int target) { + if (target == 0) { + currentDoc = target; + return target; + } else { + return NO_MORE_DOCS; + } + } + } + + private static final class MemoryByteVectorValues extends ByteVectorValues { + private final Info info; + private int currentDoc = -1; + + MemoryByteVectorValues(Info info) { + this.info = info; + } + + @Override + public int dimension() { + return info.fieldInfo.getVectorDimension(); + } + + @Override + public int size() { + return info.byteVectorCount; + } + + @Override + public byte[] vectorValue() { + if (currentDoc == 0) { + return info.byteVectorValues[0]; + } else { + return null; + } + } + + @Override + public VectorScorer scorer(byte[] query) { + if (query.length != info.fieldInfo.getVectorDimension()) { + throw new IllegalArgumentException( + "query vector dimension " + + query.length + + " does not match field dimension " + + info.fieldInfo.getVectorDimension()); + } + MemoryByteVectorValues vectorValues = new MemoryByteVectorValues(info); + return new VectorScorer() { + @Override + public float score() { + return info.fieldInfo + .getVectorSimilarityFunction() + .compare(vectorValues.vectorValue(), query); + } + + @Override + public DocIdSetIterator iterator() { + return vectorValues; + } + }; + } + + @Override + public int docID() { + return currentDoc; + } + + @Override + public int nextDoc() { + int doc = ++currentDoc; + if (doc == 0) { + return doc; + } else { + return NO_MORE_DOCS; + } + } + + @Override + public int advance(int target) { + if (target == 0) { + currentDoc = target; + return target; + } else { + return NO_MORE_DOCS; + } + } + } } diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java index b40409f1f3b..cf5628b9873 100644 --- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java +++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java @@ -43,6 +43,8 @@ import org.apache.lucene.document.FloatPoint; import org.apache.lucene.document.IntField; import org.apache.lucene.document.IntPoint; import org.apache.lucene.document.InvertableType; +import org.apache.lucene.document.KnnByteVectorField; +import org.apache.lucene.document.KnnFloatVectorField; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.SortedDocValuesField; @@ -53,8 +55,10 @@ import org.apache.lucene.document.StoredValue; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.FieldInvertState; +import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; @@ -68,6 +72,7 @@ import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; @@ -76,6 +81,7 @@ import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermStatistics; +import org.apache.lucene.search.VectorScorer; import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.search.similarities.ClassicSimilarity; import org.apache.lucene.search.similarities.Similarity; @@ -741,6 +747,170 @@ public class TestMemoryIndex extends LuceneTestCase { d, "multibinary", new BytesRef[] {new BytesRef("bbar"), new BytesRef("bbaz")}); } + public void testKnnFloatVectorOnlyOneVectorAllowed() throws IOException { + Document doc = new Document(); + doc.add(new KnnFloatVectorField("knnFloatA", new float[] {1.0f, 2.0f})); + doc.add(new KnnFloatVectorField("knnFloatA", new float[] {3.0f, 4.0f})); + expectThrows( + IllegalArgumentException.class, + () -> MemoryIndex.fromDocument(doc, new StandardAnalyzer())); + } + + public void testKnnFloatVectors() throws IOException { + List fields = new ArrayList<>(); + fields.add(new KnnFloatVectorField("knnFloatA", new float[] {1.0f, 2.0f})); + fields.add(new KnnFloatVectorField("knnFloatB", new float[] {3.0f, 4.0f, 5.0f, 6.0f})); + fields.add( + new KnnFloatVectorField( + "knnFloatC", new float[] {7.0f, 8.0f, 9.0f}, VectorSimilarityFunction.DOT_PRODUCT)); + Collections.shuffle(fields, random()); + Document doc = new Document(); + for (IndexableField f : fields) { + doc.add(f); + } + + MemoryIndex mi = MemoryIndex.fromDocument(doc, new StandardAnalyzer()); + assertFloatVectorValue(mi, "knnFloatA", new float[] {1.0f, 2.0f}); + assertFloatVectorValue(mi, "knnFloatB", new float[] {3.0f, 4.0f, 5.0f, 6.0f}); + assertFloatVectorValue(mi, "knnFloatC", new float[] {7.0f, 8.0f, 9.0f}); + + assertFloatVectorScore(mi, "knnFloatA", new float[] {1.0f, 1.0f}, 0.5f); + assertFloatVectorScore(mi, "knnFloatB", new float[] {3.0f, 3.0f, 3.0f, 3.0f}, 0.06666667f); + assertFloatVectorScore(mi, "knnFloatC", new float[] {7.0f, 7.0f, 7.0f}, 84.5f); + + assertNull( + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getFloatVectorValues("knnFloatMissing")); + assertNull( + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getByteVectorValues("knnByteVectorValue")); + } + + public void testKnnByteVectorOnlyOneVectorAllowed() throws IOException { + Document doc = new Document(); + doc.add(new KnnByteVectorField("knnByteA", new byte[] {1, 2})); + doc.add(new KnnByteVectorField("knnByteA", new byte[] {3, 4})); + expectThrows( + IllegalArgumentException.class, + () -> MemoryIndex.fromDocument(doc, new StandardAnalyzer())); + } + + public void testKnnByteVectors() throws IOException { + List fields = new ArrayList<>(); + fields.add(new KnnByteVectorField("knnByteA", new byte[] {1, 2})); + fields.add(new KnnByteVectorField("knnByteB", new byte[] {3, 4, 5, 6})); + fields.add( + new KnnByteVectorField( + "knnByteC", new byte[] {7, 8, 9}, VectorSimilarityFunction.DOT_PRODUCT)); + Collections.shuffle(fields, random()); + Document doc = new Document(); + for (IndexableField f : fields) { + doc.add(f); + } + + MemoryIndex mi = MemoryIndex.fromDocument(doc, new StandardAnalyzer()); + assertByteVectorValue(mi, "knnByteA", new byte[] {1, 2}); + assertByteVectorValue(mi, "knnByteB", new byte[] {3, 4, 5, 6}); + assertByteVectorValue(mi, "knnByteC", new byte[] {7, 8, 9}); + + assertByteVectorScore(mi, "knnByteA", new byte[] {1, 1}, 0.5f); + assertByteVectorScore(mi, "knnByteB", new byte[] {3, 3, 3, 3}, 0.06666667f); + assertByteVectorScore(mi, "knnByteC", new byte[] {7, 7, 7}, 0.501709f); + + assertNull( + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getByteVectorValues("knnByteMissing")); + assertNull( + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getFloatVectorValues("knnFloatVectorValue")); + } + + private static void assertFloatVectorValue(MemoryIndex mi, String fieldName, float[] expected) + throws IOException { + FloatVectorValues fvv = + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getFloatVectorValues(fieldName); + assertNotNull(fvv); + assertEquals(0, fvv.nextDoc()); + assertArrayEquals(expected, fvv.vectorValue(), 1e-6f); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, fvv.nextDoc()); + } + + private static void assertFloatVectorScore( + MemoryIndex mi, String fieldName, float[] queryVector, float expectedScore) + throws IOException { + FloatVectorValues fvv = + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getFloatVectorValues(fieldName); + assertNotNull(fvv); + if (random().nextBoolean()) { + fvv.nextDoc(); + } + VectorScorer scorer = fvv.scorer(queryVector); + assertEquals(0, scorer.iterator().nextDoc()); + assertEquals(expectedScore, scorer.score(), 0.0f); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc()); + } + + private static void assertByteVectorValue(MemoryIndex mi, String fieldName, byte[] expected) + throws IOException { + ByteVectorValues bvv = + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getByteVectorValues(fieldName); + assertNotNull(bvv); + assertEquals(0, bvv.nextDoc()); + assertArrayEquals(expected, bvv.vectorValue()); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, bvv.nextDoc()); + } + + private static void assertByteVectorScore( + MemoryIndex mi, String fieldName, byte[] queryVector, float expectedScore) + throws IOException { + ByteVectorValues bvv = + mi.createSearcher() + .getIndexReader() + .leaves() + .get(0) + .reader() + .getByteVectorValues(fieldName); + assertNotNull(bvv); + if (random().nextBoolean()) { + bvv.nextDoc(); + } + VectorScorer scorer = bvv.scorer(queryVector); + assertEquals(0, scorer.iterator().nextDoc()); + assertEquals(expectedScore, scorer.score(), 0.0f); + assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc()); + } + private static void assertContains( Document d, String field, Object expected, Function value) { assertNotNull(d.getField(field)); From 926d8f4ce630e7d53f674f0afbbb9294e1f3523c Mon Sep 17 00:00:00 2001 From: bjacobowitz Date: Wed, 7 Aug 2024 10:28:06 -0400 Subject: [PATCH 53/66] Make CandidateMatcher functions public (#13632) A number of functions in CandidateMatcher are protected or package-protected, meaning that client code can't use them, which makes it difficult to build custom wrapper matchers. This commit makes these functions public --- lucene/CHANGES.txt | 2 + .../lucene/monitor/CandidateMatcher.java | 6 +- .../lucene/monitor/CollectingMatcher.java | 2 +- .../lucene/monitor/ExplainingMatch.java | 4 +- .../lucene/monitor/HighlightsMatch.java | 4 +- .../lucene/monitor/ParallelMatcher.java | 2 +- .../lucene/monitor/PartitionMatcher.java | 2 +- .../lucene/monitor/QueryTimeListener.java | 2 +- .../apache/lucene/monitor/TestMonitor.java | 2 +- .../TestCandidateMatcherVisibility.java | 65 +++++++++++++++++++ 10 files changed, 79 insertions(+), 12 deletions(-) create mode 100644 lucene/monitor/src/test/org/apache/lucene/monitor/outsidepackage/TestCandidateMatcherVisibility.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f9b989d723f..bdc5ce0796e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -110,6 +110,8 @@ API Changes * GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) +* GITHUB#13632: CandidateMatcher public matching functions (Bryan Jacobowitz) + New Features --------------------- diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java index 6132e195edb..055f1b07c87 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java @@ -64,7 +64,7 @@ public abstract class CandidateMatcher { * @param metadata the query metadata * @throws IOException on IO errors */ - protected abstract void matchQuery(String queryId, Query matchQuery, Map metadata) + public abstract void matchQuery(String queryId, Query matchQuery, Map metadata) throws IOException; /** @@ -95,14 +95,14 @@ public abstract class CandidateMatcher { public abstract T resolve(T match1, T match2); /** Called by the Monitor if running a query throws an Exception */ - void reportError(String queryId, Exception e) { + public void reportError(String queryId, Exception e) { this.errors.put(queryId, e); } /** * @return the matches from this matcher */ - final MultiMatchingQueries finish(long buildTime, int queryCount) { + public final MultiMatchingQueries finish(long buildTime, int queryCount) { doFinish(); this.searchTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - searchTime, TimeUnit.NANOSECONDS); diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java index 02d8e3b6ba2..b6300dafa75 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java @@ -35,7 +35,7 @@ abstract class CollectingMatcher extends CandidateMatcher< } @Override - protected void matchQuery(final String queryId, Query matchQuery, Map metadata) + public void matchQuery(final String queryId, Query matchQuery, Map metadata) throws IOException { searcher.search(matchQuery, new MatchCollector(queryId, scoreMode)); } diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java index 1b4aa4044ef..19e124f1a3f 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java @@ -31,8 +31,8 @@ public class ExplainingMatch extends QueryMatch { searcher -> new CandidateMatcher(searcher) { @Override - protected void matchQuery( - String queryId, Query matchQuery, Map metadata) throws IOException { + public void matchQuery(String queryId, Query matchQuery, Map metadata) + throws IOException { int maxDocs = searcher.getIndexReader().maxDoc(); for (int i = 0; i < maxDocs; i++) { Explanation explanation = searcher.explain(matchQuery, i); diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java index aadcc75fb6e..0e3aea210a1 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java @@ -46,8 +46,8 @@ public class HighlightsMatch extends QueryMatch { new CandidateMatcher(searcher) { @Override - protected void matchQuery( - String queryId, Query matchQuery, Map metadata) throws IOException { + public void matchQuery(String queryId, Query matchQuery, Map metadata) + throws IOException { Weight w = searcher.createWeight( searcher.rewrite(matchQuery), ScoreMode.COMPLETE_NO_SCORES, 1); diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java index fd6d7efceb3..027343701fc 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java @@ -74,7 +74,7 @@ public class ParallelMatcher extends CandidateMatcher { } @Override - protected void matchQuery(String queryId, Query matchQuery, Map metadata) + public void matchQuery(String queryId, Query matchQuery, Map metadata) throws IOException { try { queue.put(new MatcherTask(queryId, matchQuery, metadata)); diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java index aaf1f576ecb..7a38edd74a1 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java @@ -79,7 +79,7 @@ public class PartitionMatcher extends CandidateMatcher } @Override - protected void matchQuery(String queryId, Query matchQuery, Map metadata) { + public void matchQuery(String queryId, Query matchQuery, Map metadata) { tasks.add(new MatchTask(queryId, matchQuery, metadata)); } diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java index a787724d181..b9e2b80f0d9 100644 --- a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java +++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java @@ -39,7 +39,7 @@ public interface QueryTimeListener { CandidateMatcher matcher = factory.createMatcher(searcher); return new CandidateMatcher(searcher) { @Override - protected void matchQuery(String queryId, Query matchQuery, Map metadata) + public void matchQuery(String queryId, Query matchQuery, Map metadata) throws IOException { long t = System.nanoTime(); matcher.matchQuery(queryId, matchQuery, metadata); diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java index aa5f3452764..52675c3f9a7 100644 --- a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java +++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java @@ -178,7 +178,7 @@ public class TestMonitor extends MonitorTestBase { docs -> new CandidateMatcher(docs) { @Override - protected void matchQuery( + public void matchQuery( String queryId, Query matchQuery, Map metadata) { assertEquals("value", metadata.get("key")); } diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/outsidepackage/TestCandidateMatcherVisibility.java b/lucene/monitor/src/test/org/apache/lucene/monitor/outsidepackage/TestCandidateMatcherVisibility.java new file mode 100644 index 00000000000..73e84112e38 --- /dev/null +++ b/lucene/monitor/src/test/org/apache/lucene/monitor/outsidepackage/TestCandidateMatcherVisibility.java @@ -0,0 +1,65 @@ +/* + * 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.monitor.outsidepackage; + +import java.io.IOException; +import java.util.Collections; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.memory.MemoryIndex; +import org.apache.lucene.monitor.CandidateMatcher; +import org.apache.lucene.monitor.QueryMatch; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.TermQuery; +import org.junit.Test; + +public class TestCandidateMatcherVisibility { + + private CandidateMatcher newCandidateMatcher() { + // Index and searcher for use in creating a matcher + MemoryIndex index = new MemoryIndex(); + final IndexSearcher searcher = index.createSearcher(); + return QueryMatch.SIMPLE_MATCHER.createMatcher(searcher); + } + + @Test + public void testMatchQueryVisibleOutsidePackage() throws IOException { + CandidateMatcher matcher = newCandidateMatcher(); + // This should compile from outside org.apache.lucene.monitor package + // (subpackage org.apache.lucene.monitor.outsidepackage cannot access package-private content + // from org.apache.lucene.monitor) + matcher.matchQuery("test", new TermQuery(new Term("test_field")), Collections.emptyMap()); + } + + @Test + public void testReportErrorVisibleOutsidePackage() { + CandidateMatcher matcher = newCandidateMatcher(); + // This should compile from outside org.apache.lucene.monitor package + // (subpackage org.apache.lucene.monitor.outsidepackage cannot access package-private content + // from org.apache.lucene.monitor) + matcher.reportError("test", new RuntimeException("test exception")); + } + + @Test + public void testFinishVisibleOutsidePackage() { + CandidateMatcher matcher = newCandidateMatcher(); + // This should compile from outside org.apache.lucene.monitor package + // (subpackage org.apache.lucene.monitor.outsidepackage cannot access package-private content + // from org.apache.lucene.monitor) + matcher.finish(0, 0); + } +} From d26b1521170f81bb238a05d32f7a5eea53708807 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 7 Aug 2024 13:31:00 -0400 Subject: [PATCH 54/66] Fix race condition on flush for DWPT seqNo generation (#13627) There is a tricky race condition with DWPT threads. It is possible that a flush starts by advancing the deleteQueue (in charge of creating seqNo). Thus, the referenced deleteQueue, there should be a cap on the number of actions left. However, it is possible after the advance, but before the DWPT are actually marked for flush, the DWPT gets freed and taken again to be used. To replicate this extreme behavior, see: https://github.com/apache/lucene/compare/main...benwtrent:lucene:test-replicate-and-debug-13127?expand=1 This commit will prevent DWPT from being added back to the free list if their queue has been advanced. This is because the `maxSeqNo` for that queue was created accounting only for the current number of active threads. If the thread gets passed out again and still references the already advanced queue, it is possible that seqNo actually advances past the set `maxSeqNo`. closes: https://github.com/apache/lucene/issues/13127 closes: https://github.com/apache/lucene/issues/13571 --- lucene/CHANGES.txt | 2 ++ .../org/apache/lucene/index/DocumentsWriter.java | 14 ++++++++++---- .../lucene/index/DocumentsWriterDeleteQueue.java | 2 +- .../lucene/index/DocumentsWriterPerThread.java | 4 ++++ .../lucene/index/DocumentsWriterPerThreadPool.java | 9 +++++++++ 5 files changed, 26 insertions(+), 5 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index bdc5ce0796e..572d2c6ff67 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -360,6 +360,8 @@ Bug Fixes * GITHUB#13615: Correct scalar quantization when used in conjunction with COSINE similarity. Vectors are normalized before quantization to ensure the cosine similarity is correctly calculated. (Ben Trent) +* GITHUB#13627: Fix race condition on flush for DWPT seqNo generation. (Ben Trent, Ao Li) + Other -------------------- (No changes) diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java index 7955df5630e..e32c8b20c04 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java @@ -430,10 +430,16 @@ final class DocumentsWriter implements Closeable, Accountable { } flushingDWPT = flushControl.doAfterDocument(dwpt); } finally { - if (dwpt.isFlushPending() || dwpt.isAborted()) { - dwpt.unlock(); - } else { - perThreadPool.marksAsFreeAndUnlock(dwpt); + // If a flush is occurring, we don't want to allow this dwpt to be reused + // If it is aborted, we shouldn't allow it to be reused + // If the deleteQueue is advanced, this means the maximum seqNo has been set and it cannot be + // reused + synchronized (flushControl) { + if (dwpt.isFlushPending() || dwpt.isAborted() || dwpt.isQueueAdvanced()) { + dwpt.unlock(); + } else { + perThreadPool.marksAsFreeAndUnlock(dwpt); + } } assert dwpt.isHeldByCurrentThread() == false : "we didn't release the dwpt even on abort"; } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java index 2cea3a6db0d..34c858eaf5d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java @@ -636,7 +636,7 @@ final class DocumentsWriterDeleteQueue implements Accountable, Closeable { } /** Returns true if it was advanced. */ - boolean isAdvanced() { + synchronized boolean isAdvanced() { return advanced; } } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index 3fe76a6338f..d472302f7e8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -718,6 +718,10 @@ final class DocumentsWriterPerThread implements Accountable, Lock { return flushPending.get() == Boolean.TRUE; } + boolean isQueueAdvanced() { + return deleteQueue.isAdvanced(); + } + /** Sets this DWPT as flush pending. This can only be set once. */ void setFlushPending() { flushPending.set(Boolean.TRUE); diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java index d69a71bfea5..93276c7ced4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java @@ -138,6 +138,15 @@ final class DocumentsWriterPerThreadPool implements Iterable Date: Thu, 8 Aug 2024 14:41:52 -0400 Subject: [PATCH 55/66] gh-12627: HnswGraphBuilder connects disconnected HNSW graph components (#13566) * gh-12627: HnswGraphBuilder connects disconnected HNSW graph components --- .../lucene99/Lucene99HnswVectorsReader.java | 1 + .../lucene99/Lucene99HnswVectorsWriter.java | 2 +- .../apache/lucene/util/hnsw/HnswBuilder.java | 2 +- .../util/hnsw/HnswConcurrentMergeBuilder.java | 13 +- .../lucene/util/hnsw/HnswGraphBuilder.java | 100 +++++- .../org/apache/lucene/util/hnsw/HnswUtil.java | 248 ++++++++++++++ .../lucene/util/hnsw/OnHeapHnswGraph.java | 5 +- .../BaseVectorSimilarityQueryTestCase.java | 8 +- .../apache/lucene/util/hnsw/TestHnswUtil.java | 312 ++++++++++++++++++ .../test-framework/src/java/module-info.java | 1 - .../lucene/tests/util/hnsw/HnswTestUtil.java | 132 -------- 11 files changed, 679 insertions(+), 145 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/util/hnsw/HnswUtil.java create mode 100644 lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java delete mode 100644 lucene/test-framework/src/java/org/apache/lucene/tests/util/hnsw/HnswTestUtil.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java index 35bc38571a6..f27a826e9c3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java @@ -463,6 +463,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader // unsafe; no bounds checking dataIn.seek(graphLevelNodeOffsets.get(targetIndex + graphLevelNodeIndexOffsets[level])); arcCount = dataIn.readVInt(); + assert arcCount <= currentNeighborsBuffer.length : "too many neighbors: " + arcCount; if (arcCount > 0) { currentNeighborsBuffer[0] = dataIn.readVInt(); for (int i = 1; i < arcCount; i++) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java index ce7d88f9dfd..04427a253ae 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java @@ -612,7 +612,7 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { throw new UnsupportedOperationException(); } - OnHeapHnswGraph getGraph() { + OnHeapHnswGraph getGraph() throws IOException { assert flatFieldVectorsWriter.isFinished(); if (node > 0) { return hnswGraphBuilder.getCompletedGraph(); diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java index 672bf5c6010..aa27525b7f1 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswBuilder.java @@ -48,5 +48,5 @@ public interface HnswBuilder { * components, re-ordering node ids for better delta compression) may be triggered, so callers * should expect this call to take some time. */ - OnHeapHnswGraph getCompletedGraph(); + OnHeapHnswGraph getCompletedGraph() throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java index fd1a98d08c0..74f472d78b9 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswConcurrentMergeBuilder.java @@ -91,6 +91,7 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { }); } taskExecutor.invokeAll(futures); + finish(); frozen = true; return workers[0].getCompletedGraph(); } @@ -109,11 +110,19 @@ public class HnswConcurrentMergeBuilder implements HnswBuilder { } @Override - public OnHeapHnswGraph getCompletedGraph() { - frozen = true; + public OnHeapHnswGraph getCompletedGraph() throws IOException { + if (frozen == false) { + // should already have been called in build(), but just in case + finish(); + frozen = true; + } return getGraph(); } + private void finish() throws IOException { + workers[0].finish(); + } + @Override public OnHeapHnswGraph getGraph() { return workers[0].getGraph(); diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index 6d06c1298a9..f31c7942680 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -18,8 +18,11 @@ package org.apache.lucene.util.hnsw; import static java.lang.Math.log; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; +import java.util.Comparator; +import java.util.List; import java.util.Locale; import java.util.Objects; import java.util.SplittableRandom; @@ -28,6 +31,7 @@ import org.apache.lucene.search.KnnCollector; import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.hnsw.HnswUtil.Component; /** * Builder for HNSW graph. See {@link HnswGraph} for a gloss on the algorithm and the meaning of the @@ -137,7 +141,7 @@ public class HnswGraphBuilder implements HnswBuilder { HnswGraphSearcher graphSearcher) throws IOException { if (M <= 0) { - throw new IllegalArgumentException("maxConn must be positive"); + throw new IllegalArgumentException("M (max connections) must be positive"); } if (beamWidth <= 0) { throw new IllegalArgumentException("beamWidth must be positive"); @@ -173,8 +177,11 @@ public class HnswGraphBuilder implements HnswBuilder { } @Override - public OnHeapHnswGraph getCompletedGraph() { - frozen = true; + public OnHeapHnswGraph getCompletedGraph() throws IOException { + if (!frozen) { + finish(); + frozen = true; + } return getGraph(); } @@ -405,6 +412,93 @@ public class HnswGraphBuilder implements HnswBuilder { return ((int) (-log(randDouble) * ml)); } + void finish() throws IOException { + connectComponents(); + } + + private void connectComponents() throws IOException { + long start = System.nanoTime(); + for (int level = 0; level < hnsw.numLevels(); level++) { + if (connectComponents(level) == false) { + if (infoStream.isEnabled(HNSW_COMPONENT)) { + infoStream.message(HNSW_COMPONENT, "connectComponents failed on level " + level); + } + } + } + if (infoStream.isEnabled(HNSW_COMPONENT)) { + infoStream.message( + HNSW_COMPONENT, "connectComponents " + (System.nanoTime() - start) / 1_000_000 + " ms"); + } + } + + private boolean connectComponents(int level) throws IOException { + FixedBitSet notFullyConnected = new FixedBitSet(hnsw.size()); + int maxConn = M; + if (level == 0) { + maxConn *= 2; + } + List components = HnswUtil.components(hnsw, level, notFullyConnected, maxConn); + boolean result = true; + if (components.size() > 1) { + // connect other components to the largest one + Component c0 = components.stream().max(Comparator.comparingInt(Component::size)).get(); + if (c0.start() == NO_MORE_DOCS) { + // the component is already fully connected - no room for new connections + return false; + } + // try for more connections? We only do one since otherwise they may become full + // while linking + GraphBuilderKnnCollector beam = new GraphBuilderKnnCollector(1); + int[] eps = new int[1]; + for (Component c : components) { + if (c != c0) { + beam.clear(); + eps[0] = c0.start(); + RandomVectorScorer scorer = scorerSupplier.scorer(c.start()); + // find the closest node in the largest component to the lowest-numbered node in this + // component that has room to make a connection + graphSearcher.searchLevel(beam, scorer, 0, eps, hnsw, notFullyConnected); + boolean linked = false; + while (beam.size() > 0) { + float score = beam.minimumScore(); + int c0node = beam.popNode(); + assert notFullyConnected.get(c0node); + // link the nodes + link(level, c0node, c.start(), score, notFullyConnected); + linked = true; + } + if (!linked) { + result = false; + } + } + } + } + return result; + } + + // Try to link two nodes bidirectionally; the forward connection will always be made. + // Update notFullyConnected. + private void link(int level, int n0, int n1, float score, FixedBitSet notFullyConnected) { + NeighborArray nbr0 = hnsw.getNeighbors(level, n0); + NeighborArray nbr1 = hnsw.getNeighbors(level, n1); + // must subtract 1 here since the nodes array is one larger than the configured + // max neighbors (M / 2M). + // We should have taken care of this check by searching for not-full nodes + int maxConn = nbr0.nodes().length - 1; + assert notFullyConnected.get(n0); + assert nbr0.size() < maxConn : "node " + n0 + " is full, has " + nbr0.size() + " friends"; + nbr0.addOutOfOrder(n1, score); + if (nbr0.size() == maxConn) { + notFullyConnected.clear(n0); + } + if (nbr1.size() < maxConn) { + nbr1.addOutOfOrder(n0, score); + if (nbr1.size() == maxConn) { + notFullyConnected.clear(n1); + } + } + } + /** * A restricted, specialized knnCollector that can be used when building a graph. * diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswUtil.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswUtil.java new file mode 100644 index 00000000000..bea4a6031ad --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswUtil.java @@ -0,0 +1,248 @@ +/* + * 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.hnsw; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import org.apache.lucene.codecs.hnsw.HnswGraphProvider; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.FilterLeafReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.FixedBitSet; + +/** Utilities for use in tests involving HNSW graphs */ +public class HnswUtil { + + // utility class; only has static methods + private HnswUtil() {} + + /* + For each level, check rooted components from previous level nodes, which are entry + points with the goal that each node should be reachable from *some* entry point. For each entry + point, compute a spanning tree, recording the nodes in a single shared bitset. + + Also record a bitset marking nodes that are not full to be used when reconnecting in order to + limit the search to include non-full nodes only. + */ + + /** Returns true if every node on every level is reachable from node 0. */ + static boolean isRooted(HnswGraph knnValues) throws IOException { + for (int level = 0; level < knnValues.numLevels(); level++) { + if (components(knnValues, level, null, 0).size() > 1) { + return false; + } + } + return true; + } + + /** + * Returns the sizes of the distinct graph components on level 0. If the graph is fully-rooted the + * list will have one entry. If it is empty, the returned list will be empty. + */ + static List componentSizes(HnswGraph hnsw) throws IOException { + return componentSizes(hnsw, 0); + } + + /** + * Returns the sizes of the distinct graph components on the given level. The forest starting at + * the entry points (nodes in the next highest level) is considered as a single component. If the + * entire graph is rooted in the entry points, that is every node is reachable from at least one + * entry point, the returned list will have a single entry. If the graph is empty, the returned + * list will be empty. + */ + static List componentSizes(HnswGraph hnsw, int level) throws IOException { + return components(hnsw, level, null, 0).stream().map(Component::size).toList(); + } + + // Finds orphaned components on the graph level. + static List components( + HnswGraph hnsw, int level, FixedBitSet notFullyConnected, int maxConn) throws IOException { + List components = new ArrayList<>(); + FixedBitSet connectedNodes = new FixedBitSet(hnsw.size()); + assert hnsw.size() == hnsw.getNodesOnLevel(0).size(); + int total = 0; + if (level >= hnsw.numLevels()) { + throw new IllegalArgumentException( + "Level " + level + " too large for graph with " + hnsw.numLevels() + " levels"); + } + HnswGraph.NodesIterator entryPoints; + // System.out.println("components level=" + level); + if (level == hnsw.numLevels() - 1) { + entryPoints = new HnswGraph.ArrayNodesIterator(new int[] {hnsw.entryNode()}, 1); + } else { + entryPoints = hnsw.getNodesOnLevel(level + 1); + } + while (entryPoints.hasNext()) { + int entryPoint = entryPoints.nextInt(); + Component component = + markRooted(hnsw, level, connectedNodes, notFullyConnected, maxConn, entryPoint); + total += component.size(); + } + int entryPoint; + if (notFullyConnected != null) { + entryPoint = notFullyConnected.nextSetBit(0); + } else { + entryPoint = connectedNodes.nextSetBit(0); + } + components.add(new Component(entryPoint, total)); + if (level == 0) { + int nextClear = nextClearBit(connectedNodes, 0); + while (nextClear != NO_MORE_DOCS) { + Component component = + markRooted(hnsw, level, connectedNodes, notFullyConnected, maxConn, nextClear); + assert component.size() > 0; + components.add(component); + total += component.size(); + nextClear = nextClearBit(connectedNodes, component.start()); + } + } else { + HnswGraph.NodesIterator nodes = hnsw.getNodesOnLevel(level); + while (nodes.hasNext()) { + int nextClear = nodes.nextInt(); + if (connectedNodes.get(nextClear)) { + continue; + } + Component component = + markRooted(hnsw, level, connectedNodes, notFullyConnected, maxConn, nextClear); + assert component.size() > 0; + components.add(component); + total += component.size(); + } + } + assert total == hnsw.getNodesOnLevel(level).size() + : "total=" + + total + + " level nodes on level " + + level + + " = " + + hnsw.getNodesOnLevel(level).size(); + return components; + } + + /** + * Count the nodes in a rooted component of the graph and set the bits of its nodes in + * connectedNodes bitset. Rooted means nodes that can be reached from a root node. + * + * @param hnswGraph the graph to check + * @param level the level of the graph to check + * @param connectedNodes a bitset the size of the entire graph with 1's indicating nodes that have + * been marked as connected. This method updates the bitset. + * @param notFullyConnected a bitset the size of the entire graph. On output, we mark nodes + * visited having fewer than maxConn connections. May be null. + * @param maxConn the maximum number of connections for any node (aka M). + * @param entryPoint a node id to start at + */ + private static Component markRooted( + HnswGraph hnswGraph, + int level, + FixedBitSet connectedNodes, + FixedBitSet notFullyConnected, + int maxConn, + int entryPoint) + throws IOException { + // Start at entry point and search all nodes on this level + // System.out.println("markRooted level=" + level + " entryPoint=" + entryPoint); + Deque stack = new ArrayDeque<>(); + stack.push(entryPoint); + int count = 0; + while (!stack.isEmpty()) { + int node = stack.pop(); + if (connectedNodes.get(node)) { + continue; + } + count++; + connectedNodes.set(node); + hnswGraph.seek(level, node); + int friendOrd; + int friendCount = 0; + while ((friendOrd = hnswGraph.nextNeighbor()) != NO_MORE_DOCS) { + ++friendCount; + stack.push(friendOrd); + } + if (friendCount < maxConn && notFullyConnected != null) { + notFullyConnected.set(node); + } + } + return new Component(entryPoint, count); + } + + private static int nextClearBit(FixedBitSet bits, int index) { + // Does not depend on the ghost bits being clear! + long[] barray = bits.getBits(); + assert index >= 0 && index < bits.length() : "index=" + index + ", numBits=" + bits.length(); + int i = index >> 6; + long word = ~(barray[i] >> index); // skip all the bits to the right of index + + int next = NO_MORE_DOCS; + if (word != 0) { + next = index + Long.numberOfTrailingZeros(word); + } else { + while (++i < barray.length) { + word = ~barray[i]; + if (word != 0) { + next = (i << 6) + Long.numberOfTrailingZeros(word); + break; + } + } + } + if (next >= bits.length()) { + return NO_MORE_DOCS; + } else { + return next; + } + } + + /** + * In graph theory, "connected components" are really defined only for undirected (ie + * bidirectional) graphs. Our graphs are directed, because of pruning, but they are *mostly* + * undirected. In this case we compute components starting from a single node so what we are + * really measuring is whether the graph is a "rooted graph". TODO: measure whether the graph is + * "strongly connected" ie there is a path from every node to every other node. + */ + public static boolean graphIsRooted(IndexReader reader, String vectorField) throws IOException { + for (LeafReaderContext ctx : reader.leaves()) { + CodecReader codecReader = (CodecReader) FilterLeafReader.unwrap(ctx.reader()); + HnswGraph graph = + ((HnswGraphProvider) + ((PerFieldKnnVectorsFormat.FieldsReader) codecReader.getVectorReader()) + .getFieldReader(vectorField)) + .getGraph(vectorField); + if (isRooted(graph) == false) { + return false; + } + } + return true; + } + + /** + * A component (also "connected component") of an undirected graph is a collection of nodes that + * are connected by neighbor links: every node in a connected component is reachable from every + * other node in the component. See https://en.wikipedia.org/wiki/Component_(graph_theory). Such a + * graph is said to be "fully connected" iff it has a single component, or it is empty. + * + * @param start the lowest-numbered node in the component + * @param size the number of nodes in the component + */ + record Component(int start, int size) {} +} diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java index d50c96c4872..a79bcd17d91 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java @@ -90,7 +90,10 @@ public final class OnHeapHnswGraph extends HnswGraph implements Accountable { * @param node the node whose neighbors are returned, represented as an ordinal on the level 0. */ public NeighborArray getNeighbors(int level, int node) { - assert graph[node][level] != null; + assert node < graph.length; + assert level < graph[node].length + : "level=" + level + ", node has only " + graph[node].length + " levels"; + assert graph[node][level] != null : "node=" + node + ", level=" + level; return graph[node][level]; } diff --git a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java index 1e32c07b665..c3623d9c28d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java +++ b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java @@ -41,7 +41,7 @@ import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.LuceneTestCase; -import org.apache.lucene.tests.util.hnsw.HnswTestUtil; +import org.apache.lucene.util.hnsw.HnswUtil; @LuceneTestCase.SuppressCodecs("SimpleText") abstract class BaseVectorSimilarityQueryTestCase< @@ -135,7 +135,7 @@ abstract class BaseVectorSimilarityQueryTestCase< try (Directory indexStore = getIndexStore(getRandomVectors(numDocs, dim)); IndexReader reader = DirectoryReader.open(indexStore)) { IndexSearcher searcher = newSearcher(reader); - assumeTrue("graph is disconnected", HnswTestUtil.graphIsConnected(reader, vectorField)); + assumeTrue("graph is disconnected", HnswUtil.graphIsRooted(reader, vectorField)); // All vectors are above -Infinity Query query1 = @@ -171,7 +171,7 @@ abstract class BaseVectorSimilarityQueryTestCase< try (Directory indexStore = getIndexStore(getRandomVectors(numDocs, dim)); IndexReader reader = DirectoryReader.open(indexStore)) { - assumeTrue("graph is disconnected", HnswTestUtil.graphIsConnected(reader, vectorField)); + assumeTrue("graph is disconnected", HnswUtil.graphIsRooted(reader, vectorField)); IndexSearcher searcher = newSearcher(reader); Query query = @@ -296,7 +296,7 @@ abstract class BaseVectorSimilarityQueryTestCase< w.commit(); try (IndexReader reader = DirectoryReader.open(indexStore)) { - assumeTrue("graph is disconnected", HnswTestUtil.graphIsConnected(reader, vectorField)); + assumeTrue("graph is disconnected", HnswUtil.graphIsRooted(reader, vectorField)); IndexSearcher searcher = newSearcher(reader); Query query = diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java new file mode 100644 index 00000000000..f36cdc68df4 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java @@ -0,0 +1,312 @@ +/* + * 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.hnsw; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.List; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.FixedBitSet; + +public class TestHnswUtil extends LuceneTestCase { + + public void testTreeWithCycle() throws Exception { + // test a graph that is a tree - this is rooted from its root node, not rooted + // from any other node, and not strongly connected + int[][][] nodes = { + { + {1, 2}, // node 0 + {3, 4}, // node 1 + {5, 6}, // node 2 + {}, {}, {}, {0} + } + }; + HnswGraph graph = new MockGraph(nodes); + assertTrue(HnswUtil.isRooted(graph)); + assertEquals(List.of(7), HnswUtil.componentSizes(graph)); + } + + public void testBackLinking() throws Exception { + // test a graph that is a tree - this is rooted from its root node, not rooted + // from any other node, and not strongly connected + int[][][] nodes = { + { + {1, 2}, // node 0 + {3, 4}, // node 1 + {0}, // node 2 + {1}, {1}, {1}, {1} + } + }; + HnswGraph graph = new MockGraph(nodes); + assertFalse(HnswUtil.isRooted(graph)); + // [ {0, 1, 2, 3, 4}, {5}, {6} + assertEquals(List.of(5, 1, 1), HnswUtil.componentSizes(graph)); + } + + public void testChain() throws Exception { + // test a graph that is a chain - this is rooted from every node, thus strongly connected + int[][][] nodes = {{{1}, {2}, {3}, {0}}}; + HnswGraph graph = new MockGraph(nodes); + assertTrue(HnswUtil.isRooted(graph)); + assertEquals(List.of(4), HnswUtil.componentSizes(graph)); + } + + public void testTwoChains() throws Exception { + // test a graph that is two chains + int[][][] nodes = {{{2}, {3}, {0}, {1}}}; + HnswGraph graph = new MockGraph(nodes); + assertFalse(HnswUtil.isRooted(graph)); + assertEquals(List.of(2, 2), HnswUtil.componentSizes(graph)); + } + + public void testLevels() throws Exception { + // test a graph that has three levels + int[][][] nodes = { + {{1, 2}, {3}, {0}, {0}}, + {{2}, null, {0}, null}, + {{}, null, null, null} + }; + HnswGraph graph = new MockGraph(nodes); + // System.out.println(graph.toString()); + assertTrue(HnswUtil.isRooted(graph)); + assertEquals(List.of(4), HnswUtil.componentSizes(graph)); + } + + public void testLevelsNotRooted() throws Exception { + // test a graph that has two levels with an orphaned node + int[][][] nodes = { + {{1}, {0}, {0}}, + {{}, null, null} + }; + HnswGraph graph = new MockGraph(nodes); + assertFalse(HnswUtil.isRooted(graph)); + assertEquals(List.of(2, 1), HnswUtil.componentSizes(graph)); + } + + public void testRandom() throws Exception { + for (int i = 0; i < atLeast(10); i++) { + // test on a random directed graph comparing against a brute force algorithm + int numNodes = random().nextInt(1, 100); + int numLevels = (int) Math.ceil(Math.log(numNodes)); + int[][][] nodes = new int[numLevels][][]; + for (int level = numLevels - 1; level >= 0; level--) { + nodes[level] = new int[numNodes][]; + for (int node = 0; node < numNodes; node++) { + if (level > 0) { + if ((level == numLevels - 1 && node > 0) + || (level < numLevels - 1 && nodes[level + 1][node] == null)) { + if (random().nextFloat() > Math.pow(Math.E, -level)) { + // skip some nodes, more on higher levels while ensuring every node present on a + // given level is present on all lower levels. Also ensure node 0 is always present. + continue; + } + } + } + int numNbrs = random().nextInt((numNodes + 7) / 8); + if (level == 0) { + numNbrs *= 2; + } + nodes[level][node] = new int[numNbrs]; + for (int nbr = 0; nbr < numNbrs; nbr++) { + while (true) { + int randomNbr = random().nextInt(numNodes); + if (nodes[level][randomNbr] != null) { + // allow self-linking; this doesn't arise in HNSW but it's valid more generally + nodes[level][node][nbr] = randomNbr; + break; + } + // nbr not on this level, try again + } + } + } + } + MockGraph graph = new MockGraph(nodes); + /* + System.out.println("iter " + i); + System.out.print(graph.toString()); + */ + assertEquals(isRooted(nodes), HnswUtil.isRooted(graph)); + } + } + + private boolean isRooted(int[][][] nodes) { + for (int level = nodes.length - 1; level >= 0; level--) { + if (isRooted(nodes, level) == false) { + return false; + } + } + return true; + } + + private boolean isRooted(int[][][] nodes, int level) { + // check that the graph is rooted in the union of the entry nodes' trees + // System.out.println("isRooted level=" + level); + int entryPointLevel; + if (level == nodes.length - 1) { + entryPointLevel = level; + } else { + entryPointLevel = level + 1; + } + FixedBitSet connected = new FixedBitSet(nodes[level].length); + int count = 0; + for (int entryPoint = 0; entryPoint < nodes[entryPointLevel].length; entryPoint++) { + if (nodes[entryPointLevel][entryPoint] == null) { + // use nodes present on next higher level (or this level if top level) as entry points + continue; + } + // System.out.println(" isRooted level=" + level + " entryPoint=" + entryPoint); + ArrayDeque stack = new ArrayDeque<>(); + stack.push(entryPoint); + while (!stack.isEmpty()) { + int node = stack.pop(); + if (connected.get(node)) { + continue; + } + // System.out.println(" connected node=" + node); + connected.set(node); + count++; + for (int nbr : nodes[level][node]) { + stack.push(nbr); + } + } + } + return count == levelSize(nodes[level]); + } + + static int levelSize(int[][] nodes) { + int count = 0; + for (int[] node : nodes) { + if (node != null) { + ++count; + } + } + return count; + } + + /** Empty graph value */ + static class MockGraph extends HnswGraph { + + private final int[][][] nodes; + + private int currentLevel; + private int currentNode; + private int currentNeighbor; + + MockGraph(int[][][] nodes) { + this.nodes = nodes; + } + + @Override + public int nextNeighbor() { + if (currentNeighbor >= nodes[currentLevel][currentNode].length) { + return NO_MORE_DOCS; + } else { + return nodes[currentLevel][currentNode][currentNeighbor++]; + } + } + + @Override + public void seek(int level, int target) { + assert level >= 0 && level < nodes.length; + assert target >= 0 && target < nodes[level].length + : "target out of range: " + + target + + " for level " + + level + + "; should be less than " + + nodes[level].length; + assert nodes[level][target] != null : "target " + target + " not on level " + level; + currentLevel = level; + currentNode = target; + currentNeighbor = 0; + } + + @Override + public int size() { + return nodes[0].length; + } + + @Override + public int numLevels() { + return nodes.length; + } + + @Override + public int entryNode() { + return 0; + } + + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + for (int level = nodes.length - 1; level >= 0; level--) { + buf.append("\nLEVEL ").append(level).append("\n"); + for (int node = 0; node < nodes[level].length; node++) { + if (nodes[level][node] != null) { + buf.append(" ") + .append(node) + .append(':') + .append(Arrays.toString(nodes[level][node])) + .append("\n"); + } + } + } + return buf.toString(); + } + + @Override + public NodesIterator getNodesOnLevel(int level) { + + int count = 0; + for (int i = 0; i < nodes[level].length; i++) { + if (nodes[level][i] != null) { + count++; + } + } + final int finalCount = count; + + return new NodesIterator(finalCount) { + int cur = -1; + int curCount = 0; + + @Override + public boolean hasNext() { + return curCount < finalCount; + } + + @Override + public int nextInt() { + while (curCount < finalCount) { + if (nodes[level][++cur] != null) { + curCount++; + return cur; + } + } + throw new IllegalStateException("exhausted"); + } + + @Override + public int consume(int[] dest) { + throw new UnsupportedOperationException(); + } + }; + } + } +} diff --git a/lucene/test-framework/src/java/module-info.java b/lucene/test-framework/src/java/module-info.java index 3e6311bc697..f366d1f52b7 100644 --- a/lucene/test-framework/src/java/module-info.java +++ b/lucene/test-framework/src/java/module-info.java @@ -49,7 +49,6 @@ module org.apache.lucene.test_framework { exports org.apache.lucene.tests.store; exports org.apache.lucene.tests.util.automaton; exports org.apache.lucene.tests.util.fst; - exports org.apache.lucene.tests.util.hnsw; exports org.apache.lucene.tests.util; provides org.apache.lucene.codecs.Codec with diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/hnsw/HnswTestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/hnsw/HnswTestUtil.java deleted file mode 100644 index 955665544bc..00000000000 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/hnsw/HnswTestUtil.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.tests.util.hnsw; - -import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; - -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Deque; -import java.util.List; -import org.apache.lucene.codecs.hnsw.HnswGraphProvider; -import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; -import org.apache.lucene.index.CodecReader; -import org.apache.lucene.index.FilterLeafReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.hnsw.HnswGraph; - -/** Utilities for use in tests involving HNSW graphs */ -public class HnswTestUtil { - - /** - * Returns true iff level 0 of the graph is fully connected - that is every node is reachable from - * any entry point. - */ - public static boolean isFullyConnected(HnswGraph knnValues) throws IOException { - return componentSizes(knnValues).size() < 2; - } - - /** - * Returns the sizes of the distinct graph components on level 0. If the graph is fully-connected - * there will only be a single component. If the graph is empty, the returned list will be empty. - */ - public static List componentSizes(HnswGraph hnsw) throws IOException { - List sizes = new ArrayList<>(); - FixedBitSet connectedNodes = new FixedBitSet(hnsw.size()); - assert hnsw.size() == hnsw.getNodesOnLevel(0).size(); - int total = 0; - while (total < connectedNodes.length()) { - int componentSize = traverseConnectedNodes(hnsw, connectedNodes); - assert componentSize > 0; - sizes.add(componentSize); - total += componentSize; - } - return sizes; - } - - // count the nodes in a connected component of the graph and set the bits of its nodes in - // connectedNodes bitset - private static int traverseConnectedNodes(HnswGraph hnswGraph, FixedBitSet connectedNodes) - throws IOException { - // Start at entry point and search all nodes on this level - int entryPoint = nextClearBit(connectedNodes, 0); - if (entryPoint == NO_MORE_DOCS) { - return 0; - } - Deque stack = new ArrayDeque<>(); - stack.push(entryPoint); - int count = 0; - while (!stack.isEmpty()) { - int node = stack.pop(); - if (connectedNodes.get(node)) { - continue; - } - count++; - connectedNodes.set(node); - hnswGraph.seek(0, node); - int friendOrd; - while ((friendOrd = hnswGraph.nextNeighbor()) != NO_MORE_DOCS) { - stack.push(friendOrd); - } - } - return count; - } - - private static int nextClearBit(FixedBitSet bits, int index) { - // Does not depend on the ghost bits being clear! - long[] barray = bits.getBits(); - assert index >= 0 && index < bits.length() : "index=" + index + ", numBits=" + bits.length(); - int i = index >> 6; - long word = ~(barray[i] >> index); // skip all the bits to the right of index - - if (word != 0) { - return index + Long.numberOfTrailingZeros(word); - } - - while (++i < barray.length) { - word = ~barray[i]; - if (word != 0) { - int next = (i << 6) + Long.numberOfTrailingZeros(word); - if (next >= bits.length()) { - return NO_MORE_DOCS; - } else { - return next; - } - } - } - return NO_MORE_DOCS; - } - - public static boolean graphIsConnected(IndexReader reader, String vectorField) - throws IOException { - for (LeafReaderContext ctx : reader.leaves()) { - CodecReader codecReader = (CodecReader) FilterLeafReader.unwrap(ctx.reader()); - HnswGraph graph = - ((HnswGraphProvider) - ((PerFieldKnnVectorsFormat.FieldsReader) codecReader.getVectorReader()) - .getFieldReader(vectorField)) - .getGraph(vectorField); - if (isFullyConnected(graph) == false) { - return false; - } - } - return true; - } -} From 843273e20bef3576c6cdb68655515009fd1d4bae Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Thu, 8 Aug 2024 19:56:45 -0400 Subject: [PATCH 56/66] fix TestHnswUtil when top level is disconnected; add CHANGES entry --- lucene/CHANGES.txt | 2 ++ .../apache/lucene/util/hnsw/TestHnswUtil.java | 21 +++++++++++-------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 572d2c6ff67..0c823f1e326 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -295,6 +295,8 @@ Improvements * GITHUB#13633: Add ability to read/write knn vector values to a MemoryIndex. (Ben Trent) +* GITHUB#12627: patch HNSW graphs to improve reachability of all nodes from entry points + Optimizations --------------------- diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java index f36cdc68df4..3ec133ac46e 100644 --- a/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/TestHnswUtil.java @@ -138,10 +138,12 @@ public class TestHnswUtil extends LuceneTestCase { } } MockGraph graph = new MockGraph(nodes); - /* - System.out.println("iter " + i); - System.out.print(graph.toString()); - */ + /**/ + if (i == 2) { + System.out.println("iter " + i); + System.out.print(graph.toString()); + } + /**/ assertEquals(isRooted(nodes), HnswUtil.isRooted(graph)); } } @@ -158,16 +160,17 @@ public class TestHnswUtil extends LuceneTestCase { private boolean isRooted(int[][][] nodes, int level) { // check that the graph is rooted in the union of the entry nodes' trees // System.out.println("isRooted level=" + level); - int entryPointLevel; + int[][] entryPoints; if (level == nodes.length - 1) { - entryPointLevel = level; + // entry into the top level is from a single entry point, fixed at 0 + entryPoints = new int[][] {nodes[level][0]}; } else { - entryPointLevel = level + 1; + entryPoints = nodes[level + 1]; } FixedBitSet connected = new FixedBitSet(nodes[level].length); int count = 0; - for (int entryPoint = 0; entryPoint < nodes[entryPointLevel].length; entryPoint++) { - if (nodes[entryPointLevel][entryPoint] == null) { + for (int entryPoint = 0; entryPoint < entryPoints.length; entryPoint++) { + if (entryPoints[entryPoint] == null) { // use nodes present on next higher level (or this level if top level) as entry points continue; } From ea562f6ef2b32fe6eadf57c6381d9a69acb043c7 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 9 Aug 2024 09:44:12 +0100 Subject: [PATCH 57/66] Knn(Float-->Byte)VectorField javadocs update in KnnByteVectorQuery (#13637) --- .../java/org/apache/lucene/search/KnnByteVectorQuery.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java index db5ae4a0d9d..9d6d71bc7a7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Objects; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.document.KnnFloatVectorField; +import org.apache.lucene.document.KnnByteVectorField; import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.LeafReader; @@ -52,7 +52,7 @@ public class KnnByteVectorQuery extends AbstractKnnVectorQuery { * Find the k nearest documents to the target vector according to the vectors in the * given field. target vector. * - * @param field a field that has been indexed as a {@link KnnFloatVectorField}. + * @param field a field that has been indexed as a {@link KnnByteVectorField}. * @param target the target of the search * @param k the number of documents to find * @throws IllegalArgumentException if k is less than 1 @@ -65,7 +65,7 @@ public class KnnByteVectorQuery extends AbstractKnnVectorQuery { * Find the k nearest documents to the target vector according to the vectors in the * given field. target vector. * - * @param field a field that has been indexed as a {@link KnnFloatVectorField}. + * @param field a field that has been indexed as a {@link KnnByteVectorField}. * @param target the target of the search * @param k the number of documents to find * @param filter a filter applied before the vector search From 4d04cc26a95016fcdf033a220d67e11ecd377261 Mon Sep 17 00:00:00 2001 From: Michael Froh Date: Fri, 9 Aug 2024 09:16:15 -0700 Subject: [PATCH 58/66] Get better cost estimate on MultiTermQuery over few terms (#13201) --------- Co-authored-by: Greg Miller --- lucene/CHANGES.txt | 2 + ...actMultiTermQueryConstantScoreWrapper.java | 82 +++++++++++-------- .../lucene/search/TestWildcardQuery.java | 39 +++++++++ 3 files changed, 89 insertions(+), 34 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 0c823f1e326..3538bbeaf24 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -297,6 +297,8 @@ Improvements * GITHUB#12627: patch HNSW graphs to improve reachability of all nodes from entry points +* GITHUB#13201: Better cost estimation on MultiTermQuery over few terms. (Michael Froh) + Optimizations --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java index 7a705c0ef4d..fc18b0476a7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java @@ -154,38 +154,17 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper collectedTerms) throws IOException; - private IOSupplier rewrite(LeafReaderContext context, Terms terms) - throws IOException { - assert terms != null; - - final int fieldDocCount = terms.getDocCount(); - final TermsEnum termsEnum = q.getTermsEnum(terms); - assert termsEnum != null; - - final List collectedTerms = new ArrayList<>(); - boolean collectResult = collectTerms(fieldDocCount, termsEnum, collectedTerms); - if (collectResult && collectedTerms.isEmpty()) { - return null; + private WeightOrDocIdSetIterator rewriteAsBooleanQuery( + LeafReaderContext context, List collectedTerms) throws IOException { + BooleanQuery.Builder bq = new BooleanQuery.Builder(); + for (TermAndState t : collectedTerms) { + final TermStates termStates = new TermStates(searcher.getTopReaderContext()); + termStates.register(t.state, context.ord, t.docFreq, t.totalTermFreq); + bq.add(new TermQuery(new Term(q.field, t.term), termStates), BooleanClause.Occur.SHOULD); } - return () -> { - if (collectResult) { - // build a boolean query - BooleanQuery.Builder bq = new BooleanQuery.Builder(); - for (TermAndState t : collectedTerms) { - final TermStates termStates = new TermStates(searcher.getTopReaderContext()); - termStates.register(t.state, context.ord, t.docFreq, t.totalTermFreq); - bq.add( - new TermQuery(new Term(q.field, t.term), termStates), BooleanClause.Occur.SHOULD); - } - Query q = new ConstantScoreQuery(bq.build()); - final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score()); - return new WeightOrDocIdSetIterator(weight); - } else { - // Too many terms to rewrite as a simple bq. - // Invoke rewriteInner logic to handle rewriting: - return rewriteInner(context, fieldDocCount, terms, termsEnum, collectedTerms); - } - }; + Query q = new ConstantScoreQuery(bq.build()); + final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score()); + return new WeightOrDocIdSetIterator(weight); } private boolean collectTerms(int fieldDocCount, TermsEnum termsEnum, List terms) @@ -240,9 +219,44 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper weightOrIteratorSupplier = rewrite(context, terms); - if (weightOrIteratorSupplier == null) return null; + assert terms != null; + + final int fieldDocCount = terms.getDocCount(); + final TermsEnum termsEnum = q.getTermsEnum(terms); + assert termsEnum != null; + + List collectedTerms = new ArrayList<>(); + boolean collectResult = collectTerms(fieldDocCount, termsEnum, collectedTerms); + + final long cost; + if (collectResult) { + // Return a null supplier if no query terms were in the segment: + if (collectedTerms.isEmpty()) { + return null; + } + + // TODO: Instead of replicating the cost logic of a BooleanQuery we could consider rewriting + // to a BQ eagerly at this point and delegating to its cost method (instead of lazily + // rewriting on #get). Not sure what the performance hit would be of doing this though. + long sumTermCost = 0; + for (TermAndState collectedTerm : collectedTerms) { + sumTermCost += collectedTerm.docFreq; + } + cost = sumTermCost; + } else { + cost = estimateCost(terms, q.getTermsCount()); + } + + IOSupplier weightOrIteratorSupplier = + () -> { + if (collectResult) { + return rewriteAsBooleanQuery(context, collectedTerms); + } else { + // Too many terms to rewrite as a simple bq. + // Invoke rewriteInner logic to handle rewriting: + return rewriteInner(context, fieldDocCount, terms, termsEnum, collectedTerms); + } + }; return new ScorerSupplier() { @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestWildcardQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestWildcardQuery.java index 04807eb19ae..4dfca8101fd 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestWildcardQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestWildcardQuery.java @@ -21,6 +21,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiTerms; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; @@ -418,4 +419,42 @@ public class TestWildcardQuery extends LuceneTestCase { reader.close(); dir.close(); } + + public void testCostEstimate() throws IOException { + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + for (int i = 0; i < 1000; i++) { + Document doc = new Document(); + doc.add(newStringField("body", "foo bar", Field.Store.NO)); + writer.addDocument(doc); + doc = new Document(); + doc.add(newStringField("body", "foo wuzzle", Field.Store.NO)); + writer.addDocument(doc); + doc = new Document(); + doc.add(newStringField("body", "bar " + i, Field.Store.NO)); + writer.addDocument(doc); + } + writer.flush(); + writer.forceMerge(1); + writer.close(); + + IndexReader reader = DirectoryReader.open(dir); + IndexSearcher searcher = newSearcher(reader); + LeafReaderContext lrc = reader.leaves().get(0); + + WildcardQuery query = new WildcardQuery(new Term("body", "foo*")); + Query rewritten = searcher.rewrite(query); + Weight weight = rewritten.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, 1.0f); + ScorerSupplier supplier = weight.scorerSupplier(lrc); + assertEquals(2000, supplier.cost()); // Sum the terms doc freqs + + query = new WildcardQuery(new Term("body", "bar*")); + rewritten = searcher.rewrite(query); + weight = rewritten.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, 1.0f); + supplier = weight.scorerSupplier(lrc); + assertEquals(3000, supplier.cost()); // Too many terms, assume worst-case all terms match + + reader.close(); + dir.close(); + } } From 5aa1aa98ea2cdb37d7d8d48657703a5eb7fbaefa Mon Sep 17 00:00:00 2001 From: Ankit Jain Date: Sat, 10 Aug 2024 02:20:47 +0530 Subject: [PATCH 59/66] Minor offset tracking tweak in PointRangeQuery #matches and #relate (#13599) --- .../java/org/apache/lucene/search/PointRangeQuery.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java index bfcef92f6f6..1b6d6869c19 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java @@ -128,8 +128,8 @@ public abstract class PointRangeQuery extends Query { private final ByteArrayComparator comparator = ArrayUtil.getUnsignedComparator(bytesPerDim); private boolean matches(byte[] packedValue) { - for (int dim = 0; dim < numDims; dim++) { - int offset = dim * bytesPerDim; + int offset = 0; + for (int dim = 0; dim < numDims; dim++, offset += bytesPerDim) { if (comparator.compare(packedValue, offset, lowerPoint, offset) < 0) { // Doc's value is too low, in this dimension return false; @@ -145,9 +145,9 @@ public abstract class PointRangeQuery extends Query { private Relation relate(byte[] minPackedValue, byte[] maxPackedValue) { boolean crosses = false; + int offset = 0; - for (int dim = 0; dim < numDims; dim++) { - int offset = dim * bytesPerDim; + for (int dim = 0; dim < numDims; dim++, offset += bytesPerDim) { if (comparator.compare(minPackedValue, offset, upperPoint, offset) > 0 || comparator.compare(maxPackedValue, offset, lowerPoint, offset) < 0) { From f0558ae26392e668e85e63710b9dada5a79a480f Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Sat, 10 Aug 2024 16:59:38 -0400 Subject: [PATCH 60/66] Two fixes for recently-added HnswGraphBuilder.connectComponents: (#13642) 1. properly set frozen flag to avoid re-duplicative work 2. don't try to join a node to itself --- .../org/apache/lucene/util/hnsw/HnswGraphBuilder.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index f31c7942680..8072bae462e 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -180,7 +180,6 @@ public class HnswGraphBuilder implements HnswBuilder { public OnHeapHnswGraph getCompletedGraph() throws IOException { if (!frozen) { finish(); - frozen = true; } return getGraph(); } @@ -414,6 +413,7 @@ public class HnswGraphBuilder implements HnswBuilder { void finish() throws IOException { connectComponents(); + frozen = true; } private void connectComponents() throws IOException { @@ -438,6 +438,7 @@ public class HnswGraphBuilder implements HnswBuilder { maxConn *= 2; } List components = HnswUtil.components(hnsw, level, notFullyConnected, maxConn); + // System.out.println("HnswGraphBuilder.connectComponents " + components); boolean result = true; if (components.size() > 1) { // connect other components to the largest one @@ -448,7 +449,7 @@ public class HnswGraphBuilder implements HnswBuilder { } // try for more connections? We only do one since otherwise they may become full // while linking - GraphBuilderKnnCollector beam = new GraphBuilderKnnCollector(1); + GraphBuilderKnnCollector beam = new GraphBuilderKnnCollector(2); int[] eps = new int[1]; for (Component c : components) { if (c != c0) { @@ -460,10 +461,14 @@ public class HnswGraphBuilder implements HnswBuilder { graphSearcher.searchLevel(beam, scorer, 0, eps, hnsw, notFullyConnected); boolean linked = false; while (beam.size() > 0) { - float score = beam.minimumScore(); int c0node = beam.popNode(); + if (c0node == c.start() || notFullyConnected.get(c0node) == false) { + continue; + } + float score = beam.minimumScore(); assert notFullyConnected.get(c0node); // link the nodes + // System.out.println("link " + c0 + "." + c0node + " to " + c + "." + c.start()); link(level, c0node, c.start(), score, notFullyConnected); linked = true; } From 9b481f76f77b40a5061fb3b70763e4b4a06d31d0 Mon Sep 17 00:00:00 2001 From: mrhbj <1825636423@qq.com> Date: Mon, 12 Aug 2024 22:06:48 +0800 Subject: [PATCH 61/66] Mark some fields final in Lucene99SkipWriter (#13639) --- .../lucene99/Lucene99SkipWriter.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99SkipWriter.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99SkipWriter.java index 495cf07d9af..0c8db5bd0f2 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99SkipWriter.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99SkipWriter.java @@ -46,10 +46,10 @@ import org.apache.lucene.store.IndexOutput; * uptos(position, payload). 4. start offset. */ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter { - private int[] lastSkipDoc; - private long[] lastSkipDocPointer; - private long[] lastSkipPosPointer; - private long[] lastSkipPayPointer; + private final int[] lastSkipDoc; + private final long[] lastSkipDocPointer; + private final long[] lastSkipPosPointer; + private final long[] lastSkipPayPointer; private final IndexOutput docOut; private final IndexOutput posOut; @@ -61,7 +61,7 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter { private long curPayPointer; private int curPosBufferUpto; private int curPayloadByteUpto; - private CompetitiveImpactAccumulator[] curCompetitiveFreqNorms; + private final CompetitiveImpactAccumulator[] curCompetitiveFreqNorms; private boolean fieldHasPositions; private boolean fieldHasOffsets; private boolean fieldHasPayloads; @@ -85,7 +85,12 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter { lastSkipPosPointer = new long[maxSkipLevels]; if (payOut != null) { lastSkipPayPointer = new long[maxSkipLevels]; + } else { + lastSkipPayPointer = null; } + } else { + lastSkipPosPointer = null; + lastSkipPayPointer = null; } curCompetitiveFreqNorms = new CompetitiveImpactAccumulator[maxSkipLevels]; for (int i = 0; i < maxSkipLevels; ++i) { From 304d4e7855deb39b4650d954d027ce8697873056 Mon Sep 17 00:00:00 2001 From: Egor Potemkin Date: Mon, 12 Aug 2024 15:26:59 +0100 Subject: [PATCH 62/66] Sandbox: Compute facets while collecting (#13568) This adds a new, ground-up implementation of faceting that computes aggregations while collecting. This has the following advantages over the current faceting module: 1. Allows for flexible aggregation logic instead of just "counts" in a general way (essentially makes what's available in today's "association faceting" available beyond taxonomy-based fields). 2. When aggregating beyond "counts," association value computation can be expensive. This implementation allows values to be computed only once when used across different aggregations. 3. Reduces latency by leveraging concurrency during collection (but potentially with increased overall cost). This work has been done in the sandbox module for now since it is not yet complete (the current faceting module covers use-cases this doesn't yet) and it needs time to bake to work out API and implementation rough edges. --------- Co-authored-by: Egor Potemkin Co-authored-by: Shradha Shankar Co-authored-by: Greg Miller --- lucene/CHANGES.txt | 12 + .../lucene/internal/hppc/IntLongHashMap.java | 834 +++++++++ .../apache/lucene/search/CollectorOwner.java | 78 + .../lucene/search/DoubleValuesSource.java | 65 + .../apache/lucene/search/IndexSearcher.java | 42 +- .../internal/hppc/TestIntLongHashMap.java | 699 +++++++ lucene/demo/src/java/module-info.java | 1 + .../demo/facet/SandboxFacetsExample.java | 737 ++++++++ .../apache/lucene/facet/DrillSideways.java | 298 +-- .../lucene/facet/DrillSidewaysQuery.java | 88 +- .../lucene/facet/MultiDoubleValuesSource.java | 60 + .../lucene/facet/taxonomy/FacetLabel.java | 8 + lucene/sandbox/build.gradle | 1 + lucene/sandbox/src/java/module-info.java | 7 + .../lucene/sandbox/facet/ComparableUtils.java | 261 +++ .../sandbox/facet/FacetFieldCollector.java | 59 + .../facet/FacetFieldCollectorManager.java | 53 + .../facet/FacetFieldLeafCollector.java | 76 + .../sandbox/facet/cutters/FacetCutter.java | 54 + .../facet/cutters/LeafFacetCutter.java | 34 + .../facet/cutters/LongValueFacetCutter.java | 187 ++ .../facet/cutters/TaxonomyFacetsCutter.java | 199 ++ .../sandbox/facet/cutters/package-info.java | 23 + .../ranges/DoubleRangeFacetCutter.java | 90 + .../facet/cutters/ranges/IntervalTracker.java | 95 + .../cutters/ranges/LongRangeFacetCutter.java | 320 ++++ .../facet/cutters/ranges/LongRangeNode.java | 94 + .../NonOverlappingLongRangeFacetCutter.java | 125 ++ .../OverlappingLongRangeFacetCutter.java | 273 +++ .../facet/cutters/ranges/package-info.java | 22 + .../CandidateSetOrdinalIterator.java | 63 + .../facet/iterators/ComparableSupplier.java | 43 + .../facet/iterators/OrdinalIterator.java | 69 + .../TaxonomyChildrenOrdinalIterator.java | 58 + .../facet/iterators/TopnOrdinalIterator.java | 113 ++ .../sandbox/facet/iterators/package-info.java | 22 + .../sandbox/facet/labels/LabelToOrd.java | 45 + .../sandbox/facet/labels/OrdToLabel.java | 35 + .../sandbox/facet/labels/RangeOrdToLabel.java | 52 + .../facet/labels/TaxonomyOrdLabelBiMap.java | 57 + .../sandbox/facet/labels/package-info.java | 23 + .../lucene/sandbox/facet/package-info.java | 30 + .../facet/recorders/CountFacetRecorder.java | 169 ++ .../facet/recorders/FacetRecorder.java | 63 + .../facet/recorders/LeafFacetRecorder.java | 35 + .../LongAggregationsFacetRecorder.java | 207 +++ .../facet/recorders/MultiFacetsRecorder.java | 87 + .../sandbox/facet/recorders/Reducer.java | 85 + .../sandbox/facet/recorders/package-info.java | 23 + .../sandbox/facet/SandboxFacetTestCase.java | 198 ++ .../TestCandidateSetOrdinalIterator.java | 100 + .../sandbox/facet/TestFacetRecorders.java | 478 +++++ .../sandbox/facet/TestLongValueFacet.java | 841 +++++++++ .../lucene/sandbox/facet/TestRangeFacet.java | 1647 +++++++++++++++++ .../sandbox/facet/TestTaxonomyFacet.java | 211 +++ 55 files changed, 9430 insertions(+), 219 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/internal/hppc/IntLongHashMap.java create mode 100644 lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java create mode 100644 lucene/core/src/test/org/apache/lucene/internal/hppc/TestIntLongHashMap.java create mode 100644 lucene/demo/src/java/org/apache/lucene/demo/facet/SandboxFacetsExample.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollector.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollectorManager.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldLeafCollector.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/FacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LeafFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/TaxonomyFacetsCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/package-info.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/DoubleRangeFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/IntervalTracker.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeNode.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/NonOverlappingLongRangeFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/OverlappingLongRangeFacetCutter.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/package-info.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/CandidateSetOrdinalIterator.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/ComparableSupplier.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/OrdinalIterator.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TaxonomyChildrenOrdinalIterator.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TopnOrdinalIterator.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/package-info.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/LabelToOrd.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/OrdToLabel.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdToLabel.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/TaxonomyOrdLabelBiMap.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/package-info.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/package-info.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/FacetRecorder.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LeafFacetRecorder.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/MultiFacetsRecorder.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/Reducer.java create mode 100644 lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/package-info.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/SandboxFacetTestCase.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestCandidateSetOrdinalIterator.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestFacetRecorders.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestLongValueFacet.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java create mode 100644 lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestTaxonomyFacet.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 3538bbeaf24..f78ec8ce02f 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -108,6 +108,15 @@ API Changes * GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand) +* GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and + MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar) + +* GITHUB#13568: Add CollectorOwner class that wraps CollectorManager, and handles list of Collectors and results. + Add IndexSearcher#search method that takes CollectorOwner. (Egor Potemkin) + +* GITHUB#13568: Add DrillSideways#search method that supports any collector types for any drill-sideways dimensions + or drill-down. (Egor Potemkin) + * GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) * GITHUB#13632: CandidateMatcher public matching functions (Bryan Jacobowitz) @@ -134,6 +143,8 @@ New Features DocValuesSkipper abstraction. A new flag is added to FieldType.java that configures whether to create a "skip index" for doc values. (Ignacio Vera) +* GITHUB#13568: Add sandbox facets module to compute facets while collecting. (Egor Potemkin, Shradha Shankar) + * GITHUB#13563: Add levels to doc values skip index. (Ignacio Vera) * GITHUB#13597: Align doc value skipper interval boundaries when an interval contains a constant @@ -141,6 +152,7 @@ New Features * GITHUB#13604: Add Kmeans clustering on vectors (Mayya Sharipova, Jim Ferenczi, Tom Veasey) + Improvements --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/internal/hppc/IntLongHashMap.java b/lucene/core/src/java/org/apache/lucene/internal/hppc/IntLongHashMap.java new file mode 100644 index 00000000000..9eb1ccd465a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/internal/hppc/IntLongHashMap.java @@ -0,0 +1,834 @@ +/* + * 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.internal.hppc; + +import static org.apache.lucene.internal.hppc.HashContainers.*; + +import java.util.Arrays; +import java.util.Iterator; +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.RamUsageEstimator; + +/** + * A hash map of int to long, implemented using open addressing with + * linear probing for collision resolution. + * + *

    Mostly forked and trimmed from com.carrotsearch.hppc.IntLongHashMap + * + *

    github: https://github.com/carrotsearch/hppc release 0.10.0 + * + * @lucene.internal + */ +public class IntLongHashMap + implements Iterable, Accountable, Cloneable { + + private static final long BASE_RAM_BYTES_USED = + RamUsageEstimator.shallowSizeOfInstance(IntLongHashMap.class); + + /** The array holding keys. */ + public int[] keys; + + /** The array holding values. */ + public long[] values; + + /** + * The number of stored keys (assigned key slots), excluding the special "empty" key, if any (use + * {@link #size()} instead). + * + * @see #size() + */ + protected int assigned; + + /** Mask for slot scans in {@link #keys}. */ + protected int mask; + + /** Expand (rehash) {@link #keys} when {@link #assigned} hits this value. */ + protected int resizeAt; + + /** Special treatment for the "empty slot" key marker. */ + protected boolean hasEmptyKey; + + /** The load factor for {@link #keys}. */ + protected double loadFactor; + + /** Seed used to ensure the hash iteration order is different from an iteration to another. */ + protected int iterationSeed; + + /** New instance with sane defaults. */ + public IntLongHashMap() { + this(DEFAULT_EXPECTED_ELEMENTS); + } + + /** + * New instance with sane defaults. + * + * @param expectedElements The expected number of elements guaranteed not to cause buffer + * expansion (inclusive). + */ + public IntLongHashMap(int expectedElements) { + this(expectedElements, DEFAULT_LOAD_FACTOR); + } + + /** + * New instance with the provided defaults. + * + * @param expectedElements The expected number of elements guaranteed not to cause a rehash + * (inclusive). + * @param loadFactor The load factor for internal buffers. Insane load factors (zero, full + * capacity) are rejected by {@link #verifyLoadFactor(double)}. + */ + public IntLongHashMap(int expectedElements, double loadFactor) { + this.loadFactor = verifyLoadFactor(loadFactor); + iterationSeed = ITERATION_SEED.incrementAndGet(); + ensureCapacity(expectedElements); + } + + /** Create a hash map from all key-value pairs of another container. */ + public IntLongHashMap(IntLongHashMap container) { + this(container.size()); + putAll(container); + } + + public long put(int key, long value) { + assert assigned < mask + 1; + + final int mask = this.mask; + if (((key) == 0)) { + long previousValue = hasEmptyKey ? values[mask + 1] : 0L; + hasEmptyKey = true; + values[mask + 1] = value; + return previousValue; + } else { + final int[] keys = this.keys; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + final long previousValue = values[slot]; + values[slot] = value; + return previousValue; + } + slot = (slot + 1) & mask; + } + + if (assigned == resizeAt) { + allocateThenInsertThenRehash(slot, key, value); + } else { + keys[slot] = key; + values[slot] = value; + } + + assigned++; + return 0L; + } + } + + /** + * If the specified key is not already associated with a value, associates it with the given + * value. + * + * @return {@code true} if {@code key} did not exist and {@code value} was placed in the map, + * {@code false} otherwise. + */ + public boolean putIfAbsent(int key, long value) { + int keyIndex = indexOf(key); + if (indexExists(keyIndex)) { + return false; + } else { + indexInsert(keyIndex, key, value); + return true; + } + } + + /** Puts all key/value pairs from a given iterable into this map. */ + public int putAll(Iterable iterable) { + final int count = size(); + for (IntLongCursor c : iterable) { + put(c.key, c.value); + } + return size() - count; + } + + /** + * If key exists, putValue is inserted into the map, otherwise any + * existing value is incremented by additionValue. + * + * @param key The key of the value to adjust. + * @param putValue The value to put if key does not exist. + * @param incrementValue The value to add to the existing value if key exists. + * @return Returns the current value associated with key (after changes). + */ + public long putOrAdd(int key, long putValue, long incrementValue) { + assert assigned < mask + 1; + + int keyIndex = indexOf(key); + if (indexExists(keyIndex)) { + putValue = values[keyIndex] + incrementValue; + indexReplace(keyIndex, putValue); + } else { + indexInsert(keyIndex, key, putValue); + } + return putValue; + } + + /** + * Adds incrementValue to any existing value for the given key or + * inserts incrementValue if key did not previously exist. + * + * @param key The key of the value to adjust. + * @param incrementValue The value to put or add to the existing value if key exists. + * @return Returns the current value associated with key (after changes). + */ + public long addTo(int key, long incrementValue) { + return putOrAdd(key, incrementValue, incrementValue); + } + + /** + * Remove all values at the given key. The default value for the key type is returned if the value + * does not exist in the map. + */ + public long remove(int key) { + final int mask = this.mask; + if (((key) == 0)) { + if (!hasEmptyKey) { + return 0L; + } + hasEmptyKey = false; + long previousValue = values[mask + 1]; + values[mask + 1] = 0L; + return previousValue; + } else { + final int[] keys = this.keys; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + final long previousValue = values[slot]; + shiftConflictingKeys(slot); + return previousValue; + } + slot = (slot + 1) & mask; + } + + return 0L; + } + } + + public long get(int key) { + if (((key) == 0)) { + return hasEmptyKey ? values[mask + 1] : 0L; + } else { + final int[] keys = this.keys; + final int mask = this.mask; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + return values[slot]; + } + slot = (slot + 1) & mask; + } + + return 0L; + } + } + + public long getOrDefault(int key, long defaultValue) { + if (((key) == 0)) { + return hasEmptyKey ? values[mask + 1] : defaultValue; + } else { + final int[] keys = this.keys; + final int mask = this.mask; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + return values[slot]; + } + slot = (slot + 1) & mask; + } + + return defaultValue; + } + } + + public boolean containsKey(int key) { + if (((key) == 0)) { + return hasEmptyKey; + } else { + final int[] keys = this.keys; + final int mask = this.mask; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + return true; + } + slot = (slot + 1) & mask; + } + + return false; + } + } + + public int indexOf(int key) { + final int mask = this.mask; + if (((key) == 0)) { + return hasEmptyKey ? mask + 1 : ~(mask + 1); + } else { + final int[] keys = this.keys; + int slot = hashKey(key) & mask; + + int existing; + while (!((existing = keys[slot]) == 0)) { + if (((key) == (existing))) { + return slot; + } + slot = (slot + 1) & mask; + } + + return ~slot; + } + } + + public boolean indexExists(int index) { + assert index < 0 || (index >= 0 && index <= mask) || (index == mask + 1 && hasEmptyKey); + + return index >= 0; + } + + public long indexGet(int index) { + assert index >= 0 : "The index must point at an existing key."; + assert index <= mask || (index == mask + 1 && hasEmptyKey); + + return values[index]; + } + + public long indexReplace(int index, long newValue) { + assert index >= 0 : "The index must point at an existing key."; + assert index <= mask || (index == mask + 1 && hasEmptyKey); + + long previousValue = values[index]; + values[index] = newValue; + return previousValue; + } + + public void indexInsert(int index, int key, long value) { + assert index < 0 : "The index must not point at an existing key."; + + index = ~index; + if (((key) == 0)) { + assert index == mask + 1; + values[index] = value; + hasEmptyKey = true; + } else { + assert ((keys[index]) == 0); + + if (assigned == resizeAt) { + allocateThenInsertThenRehash(index, key, value); + } else { + keys[index] = key; + values[index] = value; + } + + assigned++; + } + } + + public long indexRemove(int index) { + assert index >= 0 : "The index must point at an existing key."; + assert index <= mask || (index == mask + 1 && hasEmptyKey); + + long previousValue = values[index]; + if (index > mask) { + assert index == mask + 1; + hasEmptyKey = false; + values[index] = 0L; + } else { + shiftConflictingKeys(index); + } + return previousValue; + } + + public void clear() { + assigned = 0; + hasEmptyKey = false; + + Arrays.fill(keys, 0); + } + + public void release() { + assigned = 0; + hasEmptyKey = false; + + keys = null; + values = null; + ensureCapacity(DEFAULT_EXPECTED_ELEMENTS); + } + + public int size() { + return assigned + (hasEmptyKey ? 1 : 0); + } + + public boolean isEmpty() { + return size() == 0; + } + + @Override + public int hashCode() { + int h = hasEmptyKey ? 0xDEADBEEF : 0; + for (IntLongCursor c : this) { + h += BitMixer.mix(c.key) + BitMixer.mix(c.value); + } + return h; + } + + @Override + public boolean equals(Object obj) { + return (this == obj) + || (obj != null && getClass() == obj.getClass() && equalElements(getClass().cast(obj))); + } + + /** Return true if all keys of some other container exist in this container. */ + protected boolean equalElements(IntLongHashMap other) { + if (other.size() != size()) { + return false; + } + + for (IntLongCursor c : other) { + int key = c.key; + if (!containsKey(key) || !((c.value) == (get(key)))) { + return false; + } + } + + return true; + } + + /** + * Ensure this container can hold at least the given number of keys (entries) without resizing its + * buffers. + * + * @param expectedElements The total number of keys, inclusive. + */ + public void ensureCapacity(int expectedElements) { + if (expectedElements > resizeAt || keys == null) { + final int[] prevKeys = this.keys; + final long[] prevValues = this.values; + allocateBuffers(minBufferSize(expectedElements, loadFactor)); + if (prevKeys != null && !isEmpty()) { + rehash(prevKeys, prevValues); + } + } + } + + @Override + public long ramBytesUsed() { + return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(keys) + RamUsageEstimator.sizeOf(values); + } + + /** + * Provides the next iteration seed used to build the iteration starting slot and offset + * increment. This method does not need to be synchronized, what matters is that each thread gets + * a sequence of varying seeds. + */ + protected int nextIterationSeed() { + return iterationSeed = BitMixer.mixPhi(iterationSeed); + } + + /** An iterator implementation for {@link #iterator}. */ + private final class EntryIterator extends AbstractIterator { + private final IntLongCursor cursor; + private final int increment; + private int index; + private int slot; + + public EntryIterator() { + cursor = new IntLongCursor(); + int seed = nextIterationSeed(); + increment = iterationIncrement(seed); + slot = seed & mask; + } + + @Override + protected IntLongCursor fetch() { + final int mask = IntLongHashMap.this.mask; + while (index <= mask) { + int existing; + index++; + slot = (slot + increment) & mask; + if (!((existing = keys[slot]) == 0)) { + cursor.index = slot; + cursor.key = existing; + cursor.value = values[slot]; + return cursor; + } + } + + if (index == mask + 1 && hasEmptyKey) { + cursor.index = index; + cursor.key = 0; + cursor.value = values[index++]; + return cursor; + } + + return done(); + } + } + + @Override + public Iterator iterator() { + return new EntryIterator(); + } + + /** Returns a specialized view of the keys of this associated container. */ + public KeysContainer keys() { + return new KeysContainer(); + } + + /** A view of the keys inside this hash map. */ + public final class KeysContainer implements Iterable { + @Override + public Iterator iterator() { + return new KeysIterator(); + } + + public int size() { + return IntLongHashMap.this.size(); + } + + public int[] toArray() { + int[] array = new int[size()]; + int i = 0; + for (IntCursor cursor : this) { + array[i++] = cursor.value; + } + return array; + } + } + + /** An iterator over the set of assigned keys. */ + private final class KeysIterator extends AbstractIterator { + private final IntCursor cursor; + private final int increment; + private int index; + private int slot; + + public KeysIterator() { + cursor = new IntCursor(); + int seed = nextIterationSeed(); + increment = iterationIncrement(seed); + slot = seed & mask; + } + + @Override + protected IntCursor fetch() { + final int mask = IntLongHashMap.this.mask; + while (index <= mask) { + int existing; + index++; + slot = (slot + increment) & mask; + if (!((existing = keys[slot]) == 0)) { + cursor.index = slot; + cursor.value = existing; + return cursor; + } + } + + if (index == mask + 1 && hasEmptyKey) { + cursor.index = index++; + cursor.value = 0; + return cursor; + } + + return done(); + } + } + + /** + * @return Returns a container with all values stored in this map. + */ + public ValuesContainer values() { + return new ValuesContainer(); + } + + /** A view over the set of values of this map. */ + public final class ValuesContainer implements Iterable { + @Override + public Iterator iterator() { + return new ValuesIterator(); + } + + public long[] toArray() { + long[] array = new long[size()]; + int i = 0; + for (LongCursor cursor : this) { + array[i++] = cursor.value; + } + return array; + } + } + + /** An iterator over the set of assigned values. */ + private final class ValuesIterator extends AbstractIterator { + private final LongCursor cursor; + private final int increment; + private int index; + private int slot; + + public ValuesIterator() { + cursor = new LongCursor(); + int seed = nextIterationSeed(); + increment = iterationIncrement(seed); + slot = seed & mask; + } + + @Override + protected LongCursor fetch() { + final int mask = IntLongHashMap.this.mask; + while (index <= mask) { + index++; + slot = (slot + increment) & mask; + if (!((keys[slot]) == 0)) { + cursor.index = slot; + cursor.value = values[slot]; + return cursor; + } + } + + if (index == mask + 1 && hasEmptyKey) { + cursor.index = index; + cursor.value = values[index++]; + return cursor; + } + + return done(); + } + } + + @Override + public IntLongHashMap clone() { + try { + + IntLongHashMap cloned = (IntLongHashMap) super.clone(); + cloned.keys = keys.clone(); + cloned.values = values.clone(); + cloned.hasEmptyKey = hasEmptyKey; + cloned.iterationSeed = ITERATION_SEED.incrementAndGet(); + return cloned; + } catch (CloneNotSupportedException e) { + throw new RuntimeException(e); + } + } + + /** Convert the contents of this map to a human-friendly string. */ + @Override + public String toString() { + final StringBuilder buffer = new StringBuilder(); + buffer.append("["); + + boolean first = true; + for (IntLongCursor cursor : this) { + if (!first) { + buffer.append(", "); + } + buffer.append(cursor.key); + buffer.append("=>"); + buffer.append(cursor.value); + first = false; + } + buffer.append("]"); + return buffer.toString(); + } + + /** Creates a hash map from two index-aligned arrays of key-value pairs. */ + public static IntLongHashMap from(int[] keys, long[] values) { + if (keys.length != values.length) { + throw new IllegalArgumentException( + "Arrays of keys and values must have an identical length."); + } + + IntLongHashMap map = new IntLongHashMap(keys.length); + for (int i = 0; i < keys.length; i++) { + map.put(keys[i], values[i]); + } + + return map; + } + + /** + * Returns a hash code for the given key. + * + *

    The output from this function should evenly distribute keys across the entire integer range. + */ + protected int hashKey(int key) { + assert !((key) == 0); // Handled as a special case (empty slot marker). + return BitMixer.mixPhi(key); + } + + /** + * Validate load factor range and return it. Override and suppress if you need insane load + * factors. + */ + protected double verifyLoadFactor(double loadFactor) { + checkLoadFactor(loadFactor, MIN_LOAD_FACTOR, MAX_LOAD_FACTOR); + return loadFactor; + } + + /** Rehash from old buffers to new buffers. */ + protected void rehash(int[] fromKeys, long[] fromValues) { + assert fromKeys.length == fromValues.length + && HashContainers.checkPowerOfTwo(fromKeys.length - 1); + + // Rehash all stored key/value pairs into the new buffers. + final int[] keys = this.keys; + final long[] values = this.values; + final int mask = this.mask; + int existing; + + // Copy the zero element's slot, then rehash everything else. + int from = fromKeys.length - 1; + keys[keys.length - 1] = fromKeys[from]; + values[values.length - 1] = fromValues[from]; + while (--from >= 0) { + if (!((existing = fromKeys[from]) == 0)) { + int slot = hashKey(existing) & mask; + while (!((keys[slot]) == 0)) { + slot = (slot + 1) & mask; + } + keys[slot] = existing; + values[slot] = fromValues[from]; + } + } + } + + /** + * Allocate new internal buffers. This method attempts to allocate and assign internal buffers + * atomically (either allocations succeed or not). + */ + protected void allocateBuffers(int arraySize) { + assert Integer.bitCount(arraySize) == 1; + + // Ensure no change is done if we hit an OOM. + int[] prevKeys = this.keys; + long[] prevValues = this.values; + try { + int emptyElementSlot = 1; + this.keys = (new int[arraySize + emptyElementSlot]); + this.values = (new long[arraySize + emptyElementSlot]); + } catch (OutOfMemoryError e) { + this.keys = prevKeys; + this.values = prevValues; + throw new BufferAllocationException( + "Not enough memory to allocate buffers for rehashing: %,d -> %,d", + e, this.mask + 1, arraySize); + } + + this.resizeAt = expandAtCount(arraySize, loadFactor); + this.mask = arraySize - 1; + } + + /** + * This method is invoked when there is a new key/ value pair to be inserted into the buffers but + * there is not enough empty slots to do so. + * + *

    New buffers are allocated. If this succeeds, we know we can proceed with rehashing so we + * assign the pending element to the previous buffer (possibly violating the invariant of having + * at least one empty slot) and rehash all keys, substituting new buffers at the end. + */ + protected void allocateThenInsertThenRehash(int slot, int pendingKey, long pendingValue) { + assert assigned == resizeAt && ((keys[slot]) == 0) && !((pendingKey) == 0); + + // Try to allocate new buffers first. If we OOM, we leave in a consistent state. + final int[] prevKeys = this.keys; + final long[] prevValues = this.values; + allocateBuffers(nextBufferSize(mask + 1, size(), loadFactor)); + assert this.keys.length > prevKeys.length; + + // We have succeeded at allocating new data so insert the pending key/value at + // the free slot in the old arrays before rehashing. + prevKeys[slot] = pendingKey; + prevValues[slot] = pendingValue; + + // Rehash old keys, including the pending key. + rehash(prevKeys, prevValues); + } + + /** + * Shift all the slot-conflicting keys and values allocated to (and including) slot. + */ + protected void shiftConflictingKeys(int gapSlot) { + final int[] keys = this.keys; + final long[] values = this.values; + final int mask = this.mask; + + // Perform shifts of conflicting keys to fill in the gap. + int distance = 0; + while (true) { + final int slot = (gapSlot + (++distance)) & mask; + final int existing = keys[slot]; + if (((existing) == 0)) { + break; + } + + final int idealSlot = hashKey(existing); + final int shift = (slot - idealSlot) & mask; + if (shift >= distance) { + // Entry at this position was originally at or before the gap slot. + // Move the conflict-shifted entry to the gap's position and repeat the procedure + // for any entries to the right of the current position, treating it + // as the new gap. + keys[gapSlot] = existing; + values[gapSlot] = values[slot]; + gapSlot = slot; + distance = 0; + } + } + + // Mark the last found gap slot without a conflict as empty. + keys[gapSlot] = 0; + values[gapSlot] = 0L; + assigned--; + } + + /** Forked from HPPC, holding int index,key and value */ + public static final class IntLongCursor { + /** + * The current key and value's index in the container this cursor belongs to. The meaning of + * this index is defined by the container (usually it will be an index in the underlying storage + * buffer). + */ + public int index; + + /** The current key. */ + public int key; + + /** The current value. */ + public long value; + + @Override + public String toString() { + return "[cursor, index: " + index + ", key: " + key + ", value: " + value + "]"; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java b/lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java new file mode 100644 index 00000000000..4c5c2f1eea1 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java @@ -0,0 +1,78 @@ +/* + * 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.search; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * This class wraps {@link CollectorManager} and owns the collectors the manager creates. It is + * convenient that clients of the class don't have to worry about keeping the list of collectors, as + * well as about making the collector's type (C) compatible when reduce is called. Instances of this + * class cache results of {@link CollectorManager#reduce(Collection)}. + * + *

    Note that instance of this class ignores any {@link Collector} created by {@link + * CollectorManager#newCollector()} directly, not through {@link #newCollector()} + * + * @lucene.experimental + */ +public final class CollectorOwner { + + private final CollectorManager manager; + + private T result; + private boolean reduced; + + // TODO: For IndexSearcher, the list doesn't have to be synchronized + // because we create new collectors sequentially. Drill sideways creates new collectors in + // DrillSidewaysQuery#Weight#bulkScorer which is already called concurrently. + // I think making the list synchronized here is not a huge concern, at the same time, do we want + // to do something about it? + // e.g. have boolean property in constructor that makes it threads friendly when set? + private final List collectors = Collections.synchronizedList(new ArrayList<>()); + + public CollectorOwner(CollectorManager manager) { + this.manager = manager; + } + + /** Return a new {@link Collector}. This must return a different instance on each call. */ + public C newCollector() throws IOException { + C collector = manager.newCollector(); + collectors.add(collector); + return collector; + } + + public C getCollector(int i) { + return collectors.get(i); + } + + /** + * Returns result of {@link CollectorManager#reduce(Collection)}. The result is cached. + * + *

    This method is NOT threadsafe. + */ + public T getResult() throws IOException { + if (reduced == false) { + result = manager.reduce(collectors); + reduced = true; + } + return result; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java index 000c4b2c315..2650fb164cb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java +++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java @@ -26,6 +26,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.search.comparators.DoubleComparator; +import org.apache.lucene.util.NumericUtils; /** * Base class for producing {@link DoubleValues} @@ -115,6 +116,70 @@ public abstract class DoubleValuesSource implements SegmentCacheable { return new LongDoubleValuesSource(this); } + /** Convert to {@link LongValuesSource} by calling {@link NumericUtils#doubleToSortableLong} */ + public final LongValuesSource toSortableLongDoubleValuesSource() { + return new SortableLongDoubleValuesSource(this); + } + + private static class SortableLongDoubleValuesSource extends LongValuesSource { + + private final DoubleValuesSource inner; + + private SortableLongDoubleValuesSource(DoubleValuesSource inner) { + this.inner = Objects.requireNonNull(inner); + } + + @Override + public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException { + DoubleValues in = inner.getValues(ctx, scores); + + return new LongValues() { + @Override + public long longValue() throws IOException { + return NumericUtils.doubleToSortableLong(in.doubleValue()); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + return in.advanceExact(doc); + } + }; + } + + @Override + public boolean needsScores() { + return inner.needsScores(); + } + + @Override + public int hashCode() { + return inner.hashCode(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SortableLongDoubleValuesSource that = (SortableLongDoubleValuesSource) o; + return Objects.equals(inner, that.inner); + } + + @Override + public String toString() { + return "sortableLong(" + inner.toString() + ")"; + } + + @Override + public LongValuesSource rewrite(IndexSearcher searcher) throws IOException { + return inner.rewrite(searcher).toLongValuesSource(); + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return false; + } + } + private static class LongDoubleValuesSource extends LongValuesSource { private final DoubleValuesSource inner; diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 77d6edf34a0..6842d214236 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -630,27 +630,42 @@ public class IndexSearcher { */ public T search(Query query, CollectorManager collectorManager) throws IOException { - final C firstCollector = collectorManager.newCollector(); + CollectorOwner collectorOwner = new CollectorOwner<>(collectorManager); + final C firstCollector = collectorOwner.newCollector(); query = rewrite(query, firstCollector.scoreMode().needsScores()); final Weight weight = createWeight(query, firstCollector.scoreMode(), 1); - return search(weight, collectorManager, firstCollector); + search(weight, collectorOwner, firstCollector); + return collectorOwner.getResult(); } - private T search( - Weight weight, CollectorManager collectorManager, C firstCollector) throws IOException { + /** + * Lower-level search API. Search all leaves using the given {@link CollectorOwner}, without + * calling {@link CollectorOwner#getResult()} so that clients can reduce and read results + * themselves. + * + *

    Note that this method doesn't return anything - users can access results by calling {@link + * CollectorOwner#getResult()} + * + * @lucene.experimental + */ + public void search(Query query, CollectorOwner collectorOwner) + throws IOException { + final C firstCollector = collectorOwner.newCollector(); + query = rewrite(query, firstCollector.scoreMode().needsScores()); + final Weight weight = createWeight(query, firstCollector.scoreMode(), 1); + search(weight, collectorOwner, firstCollector); + } + + private void search( + Weight weight, CollectorOwner collectorOwner, C firstCollector) throws IOException { final LeafSlice[] leafSlices = getSlices(); if (leafSlices.length == 0) { - // there are no segments, nothing to offload to the executor, but we do need to call reduce to - // create some kind of empty result + // there are no segments, nothing to offload to the executor assert leafContexts.isEmpty(); - return collectorManager.reduce(Collections.singletonList(firstCollector)); } else { - final List collectors = new ArrayList<>(leafSlices.length); - collectors.add(firstCollector); final ScoreMode scoreMode = firstCollector.scoreMode(); for (int i = 1; i < leafSlices.length; ++i) { - final C collector = collectorManager.newCollector(); - collectors.add(collector); + final C collector = collectorOwner.newCollector(); if (scoreMode != collector.scoreMode()) { throw new IllegalStateException( "CollectorManager does not always produce collectors with the same score mode"); @@ -659,15 +674,14 @@ public class IndexSearcher { final List> listTasks = new ArrayList<>(leafSlices.length); for (int i = 0; i < leafSlices.length; ++i) { final LeafReaderContext[] leaves = leafSlices[i].leaves; - final C collector = collectors.get(i); + final C collector = collectorOwner.getCollector(i); listTasks.add( () -> { search(Arrays.asList(leaves), weight, collector); return collector; }); } - List results = taskExecutor.invokeAll(listTasks); - return collectorManager.reduce(results); + taskExecutor.invokeAll(listTasks); } } diff --git a/lucene/core/src/test/org/apache/lucene/internal/hppc/TestIntLongHashMap.java b/lucene/core/src/test/org/apache/lucene/internal/hppc/TestIntLongHashMap.java new file mode 100644 index 00000000000..2af50c90800 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/internal/hppc/TestIntLongHashMap.java @@ -0,0 +1,699 @@ +/* + * 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.internal.hppc; + +import com.carrotsearch.randomizedtesting.RandomizedTest; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.junit.After; +import org.junit.Test; + +/** + * Tests for {@link IntLongHashMap}. + * + *

    Mostly forked and trimmed from com.carrotsearch.hppc.IntLongHashMapTest + * + *

    github: https://github.com/carrotsearch/hppc release: 0.10.0 + */ +public class TestIntLongHashMap extends LuceneTestCase { + + /* Ready to use key values. */ + + protected int keyE = 0; + protected int key0 = cast(0), k0 = key0; + protected int key1 = cast(1), k1 = key1; + protected int key2 = cast(2), k2 = key2; + protected int key3 = cast(3), k3 = key3; + protected int key4 = cast(4), k4 = key4; + protected int key5 = cast(5), k5 = key5; + protected int key6 = cast(6), k6 = key6; + protected int key7 = cast(7), k7 = key7; + protected int key8 = cast(8), k8 = key8; + protected int key9 = cast(9), k9 = key9; + + protected long value0 = vcast(0); + protected long value1 = vcast(1); + protected long value2 = vcast(2); + protected long value3 = vcast(3); + protected long value4 = vcast(4); + + private static int randomIntBetween(int min, int max) { + return min + random().nextInt(max + 1 - min); + } + + private final int[] newArray(int... elements) { + return elements; + } + + /** Create a new array of a given type and copy the arguments to this array. */ + /* */ + private final long[] newvArray(long... elements) { + return elements; + } + + /** Convert to target type from an integer used to test stuff. */ + private int cast(Integer v) { + return v.intValue(); + } + + /** Convert to target type from an integer used to test stuff. */ + private long vcast(int value) { + return (long) value; + } + + /** Check if the array's content is identical to a given sequence of elements. */ + public static void assertSortedListEquals(int[] array, int... elements) { + assertEquals(elements.length, array.length); + Arrays.sort(array); + Arrays.sort(elements); + assertArrayEquals(elements, array); + } + + /** Check if the array's content is identical to a given sequence of elements. */ + public static void assertSortedListEquals(long[] array, long... elements) { + assertEquals(elements.length, array.length); + Arrays.sort(array); + assertArrayEquals(elements, array); + } + + /** Per-test fresh initialized instance. */ + public IntLongHashMap map = newInstance(); + + protected IntLongHashMap newInstance() { + return new IntLongHashMap(); + } + + @After + public void checkEmptySlotsUninitialized() { + if (map != null) { + int occupied = 0; + for (int i = 0; i <= map.mask; i++) { + if (((map.keys[i]) == 0)) { + + } else { + occupied++; + } + } + assertEquals(occupied, map.assigned); + + if (!map.hasEmptyKey) {} + } + } + + private void assertSameMap(final IntLongHashMap c1, final IntLongHashMap c2) { + assertEquals(c1.size(), c2.size()); + + for (IntLongHashMap.IntLongCursor entry : c1) { + assertTrue(c2.containsKey(entry.key)); + assertEquals(entry.value, c2.get(entry.key)); + } + } + + /* */ + @Test + public void testEnsureCapacity() { + final AtomicInteger expands = new AtomicInteger(); + IntLongHashMap map = + new IntLongHashMap(0) { + @Override + protected void allocateBuffers(int arraySize) { + super.allocateBuffers(arraySize); + expands.incrementAndGet(); + } + }; + + // Add some elements. + final int max = rarely() ? 0 : randomIntBetween(0, 250); + for (int i = 0; i < max; i++) { + map.put(cast(i), value0); + } + + final int additions = randomIntBetween(max, max + 5000); + map.ensureCapacity(additions + map.size()); + final int before = expands.get(); + for (int i = 0; i < additions; i++) { + map.put(cast(i), value0); + } + assertEquals(before, expands.get()); + } + + @Test + public void testCursorIndexIsValid() { + map.put(keyE, value1); + map.put(key1, value2); + map.put(key2, value3); + + for (IntLongHashMap.IntLongCursor c : map) { + assertTrue(map.indexExists(c.index)); + assertEquals(c.value, map.indexGet(c.index)); + } + } + + @Test + public void testIndexMethods() { + map.put(keyE, value1); + map.put(key1, value2); + + assertTrue(map.indexOf(keyE) >= 0); + assertTrue(map.indexOf(key1) >= 0); + assertTrue(map.indexOf(key2) < 0); + + assertTrue(map.indexExists(map.indexOf(keyE))); + assertTrue(map.indexExists(map.indexOf(key1))); + assertFalse(map.indexExists(map.indexOf(key2))); + + assertEquals(value1, map.indexGet(map.indexOf(keyE))); + assertEquals(value2, map.indexGet(map.indexOf(key1))); + + expectThrows( + AssertionError.class, + () -> { + map.indexGet(map.indexOf(key2)); + }); + + assertEquals(value1, map.indexReplace(map.indexOf(keyE), value3)); + assertEquals(value2, map.indexReplace(map.indexOf(key1), value4)); + assertEquals(value3, map.indexGet(map.indexOf(keyE))); + assertEquals(value4, map.indexGet(map.indexOf(key1))); + + map.indexInsert(map.indexOf(key2), key2, value1); + assertEquals(value1, map.indexGet(map.indexOf(key2))); + assertEquals(3, map.size()); + + assertEquals(value3, map.indexRemove(map.indexOf(keyE))); + assertEquals(2, map.size()); + assertEquals(value1, map.indexRemove(map.indexOf(key2))); + assertEquals(1, map.size()); + assertTrue(map.indexOf(keyE) < 0); + assertTrue(map.indexOf(key1) >= 0); + assertTrue(map.indexOf(key2) < 0); + } + + /* */ + @Test + public void testCloningConstructor() { + map.put(key1, value1); + map.put(key2, value2); + map.put(key3, value3); + + assertSameMap(map, new IntLongHashMap(map)); + } + + /* */ + @Test + public void testFromArrays() { + map.put(key1, value1); + map.put(key2, value2); + map.put(key3, value3); + + IntLongHashMap map2 = + IntLongHashMap.from(newArray(key1, key2, key3), newvArray(value1, value2, value3)); + + assertSameMap(map, map2); + } + + @Test + public void testGetOrDefault() { + map.put(key2, value2); + assertTrue(map.containsKey(key2)); + + map.put(key1, value1); + assertEquals(value1, map.getOrDefault(key1, value3)); + assertEquals(value3, map.getOrDefault(key3, value3)); + map.remove(key1); + assertEquals(value3, map.getOrDefault(key1, value3)); + } + + /* */ + @Test + public void testPut() { + map.put(key1, value1); + + assertTrue(map.containsKey(key1)); + assertEquals(value1, map.get(key1)); + + map.put(key2, 0L); + + assertEquals(2, map.size()); + assertTrue(map.containsKey(key2)); + assertEquals(0L, map.get(key2)); + } + + /* */ + @Test + public void testPutOverExistingKey() { + map.put(key1, value1); + assertEquals(value1, map.put(key1, value3)); + assertEquals(value3, map.get(key1)); + assertEquals(1, map.size()); + + assertEquals(value3, map.put(key1, 0L)); + assertTrue(map.containsKey(key1)); + assertEquals(0L, map.get(key1)); + + assertEquals(0L, map.put(key1, value1)); + assertEquals(value1, map.get(key1)); + assertEquals(1, map.size()); + } + + /* */ + @Test + public void testPutWithExpansions() { + final int COUNT = 10000; + final Random rnd = new Random(random().nextLong()); + final HashSet values = new HashSet(); + + for (int i = 0; i < COUNT; i++) { + final int v = rnd.nextInt(); + final boolean hadKey = values.contains(cast(v)); + values.add(cast(v)); + + assertEquals(hadKey, map.containsKey(cast(v))); + map.put(cast(v), vcast(v)); + assertEquals(values.size(), map.size()); + } + assertEquals(values.size(), map.size()); + } + + /* */ + @Test + public void testPutAll() { + map.put(key1, value1); + map.put(key2, value1); + + IntLongHashMap map2 = newInstance(); + + map2.put(key2, value2); + map2.put(keyE, value1); + + // One new key (keyE). + assertEquals(1, map.putAll(map2)); + + // Assert the value under key2 has been replaced. + assertEquals(value2, map.get(key2)); + + // And key3 has been added. + assertEquals(value1, map.get(keyE)); + assertEquals(3, map.size()); + } + + /* */ + @Test + public void testPutIfAbsent() { + assertTrue(map.putIfAbsent(key1, value1)); + assertFalse(map.putIfAbsent(key1, value2)); + assertEquals(value1, map.get(key1)); + } + + @Test + public void testPutOrAdd() { + assertEquals(value1, map.putOrAdd(key1, value1, value2)); + assertEquals(value3, map.putOrAdd(key1, value1, value2)); + } + + @Test + public void testAddTo() { + assertEquals(value1, map.addTo(key1, value1)); + assertEquals(value3, map.addTo(key1, value2)); + } + + /* */ + @Test + public void testRemove() { + map.put(key1, value1); + assertEquals(value1, map.remove(key1)); + assertEquals(0L, map.remove(key1)); + assertEquals(0, map.size()); + + // These are internals, but perhaps worth asserting too. + assertEquals(0, map.assigned); + } + + /* */ + @Test + public void testEmptyKey() { + final int empty = 0; + + map.put(empty, value1); + assertEquals(1, map.size()); + assertEquals(false, map.isEmpty()); + assertEquals(value1, map.get(empty)); + assertEquals(value1, map.getOrDefault(empty, value2)); + assertEquals(true, map.iterator().hasNext()); + assertEquals(empty, map.iterator().next().key); + assertEquals(value1, map.iterator().next().value); + + assertEquals(1, map.keys().size()); + assertEquals(empty, map.keys().iterator().next().value); + assertEquals(value1, map.values().iterator().next().value); + + assertEquals(value1, map.put(empty, 0L)); + assertEquals(1, map.size()); + assertTrue(map.containsKey(empty)); + assertEquals(0L, map.get(empty)); + + map.remove(empty); + assertEquals(0L, map.get(empty)); + assertEquals(0, map.size()); + + assertEquals(0L, map.put(empty, value1)); + assertEquals(value1, map.put(empty, value2)); + map.clear(); + assertFalse(map.indexExists(map.indexOf(empty))); + assertEquals(0L, map.put(empty, value1)); + map.clear(); + assertEquals(0L, map.remove(empty)); + } + + /* */ + @Test + public void testMapKeySet() { + map.put(key1, value3); + map.put(key2, value2); + map.put(key3, value1); + + assertSortedListEquals(map.keys().toArray(), key1, key2, key3); + } + + /* */ + @Test + public void testMapKeySetIterator() { + map.put(key1, value3); + map.put(key2, value2); + map.put(key3, value1); + + int counted = 0; + for (IntCursor c : map.keys()) { + assertEquals(map.keys[c.index], c.value); + counted++; + } + assertEquals(counted, map.size()); + } + + /* */ + @Test + public void testClear() { + map.put(key1, value1); + map.put(key2, value1); + map.clear(); + assertEquals(0, map.size()); + + // These are internals, but perhaps worth asserting too. + assertEquals(0, map.assigned); + + // Check values are cleared. + assertEquals(0L, map.put(key1, value1)); + assertEquals(0L, map.remove(key2)); + map.clear(); + + // Check if the map behaves properly upon subsequent use. + testPutWithExpansions(); + } + + /* */ + @Test + public void testRelease() { + map.put(key1, value1); + map.put(key2, value1); + map.release(); + assertEquals(0, map.size()); + + // These are internals, but perhaps worth asserting too. + assertEquals(0, map.assigned); + + // Check if the map behaves properly upon subsequent use. + testPutWithExpansions(); + } + + /* */ + @Test + public void testIterable() { + map.put(key1, value1); + map.put(key2, value2); + map.put(key3, value3); + map.remove(key2); + + int count = 0; + for (IntLongHashMap.IntLongCursor cursor : map) { + count++; + assertTrue(map.containsKey(cursor.key)); + assertEquals(cursor.value, map.get(cursor.key)); + + assertEquals(cursor.value, map.values[cursor.index]); + assertEquals(cursor.key, map.keys[cursor.index]); + } + assertEquals(count, map.size()); + + map.clear(); + assertFalse(map.iterator().hasNext()); + } + + /* */ + @Test + public void testBug_HPPC73_FullCapacityGet() { + final AtomicInteger reallocations = new AtomicInteger(); + final int elements = 0x7F; + map = + new IntLongHashMap(elements, 1f) { + @Override + protected double verifyLoadFactor(double loadFactor) { + // Skip load factor sanity range checking. + return loadFactor; + } + + @Override + protected void allocateBuffers(int arraySize) { + super.allocateBuffers(arraySize); + reallocations.incrementAndGet(); + } + }; + + int reallocationsBefore = reallocations.get(); + assertEquals(reallocationsBefore, 1); + for (int i = 1; i <= elements; i++) { + map.put(cast(i), value1); + } + + // Non-existent key. + int outOfSet = cast(elements + 1); + map.remove(outOfSet); + assertFalse(map.containsKey(outOfSet)); + assertEquals(reallocationsBefore, reallocations.get()); + + // Should not expand because we're replacing an existing element. + map.put(k1, value2); + assertEquals(reallocationsBefore, reallocations.get()); + + // Remove from a full map. + map.remove(k1); + assertEquals(reallocationsBefore, reallocations.get()); + map.put(k1, value2); + + // Check expand on "last slot of a full map" condition. + map.put(outOfSet, value1); + assertEquals(reallocationsBefore + 1, reallocations.get()); + } + + @Test + public void testHashCodeEquals() { + IntLongHashMap l0 = newInstance(); + assertEquals(0, l0.hashCode()); + assertEquals(l0, newInstance()); + + IntLongHashMap l1 = + IntLongHashMap.from(newArray(key1, key2, key3), newvArray(value1, value2, value3)); + + IntLongHashMap l2 = + IntLongHashMap.from(newArray(key2, key1, key3), newvArray(value2, value1, value3)); + + IntLongHashMap l3 = IntLongHashMap.from(newArray(key1, key2), newvArray(value2, value1)); + + assertEquals(l1.hashCode(), l2.hashCode()); + assertEquals(l1, l2); + + assertFalse(l1.equals(l3)); + assertFalse(l2.equals(l3)); + } + + @Test + public void testBug_HPPC37() { + IntLongHashMap l1 = IntLongHashMap.from(newArray(key1), newvArray(value1)); + + IntLongHashMap l2 = IntLongHashMap.from(newArray(key2), newvArray(value1)); + + assertFalse(l1.equals(l2)); + assertFalse(l2.equals(l1)); + } + + @Test + public void testEmptyValue() { + assertEquals(0L, map.put(key1, 0L)); + assertEquals(0L, map.get(key1)); + assertTrue(map.containsKey(key1)); + map.remove(key1); + assertFalse(map.containsKey(key1)); + assertEquals(0, map.size()); + } + + /** Runs random insertions/deletions/clearing and compares the results against {@link HashMap}. */ + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testAgainstHashMap() { + final Random rnd = RandomizedTest.getRandom(); + final HashMap other = new HashMap(); + + for (int size = 1000; size < 20000; size += 4000) { + other.clear(); + map.clear(); + + for (int round = 0; round < size * 20; round++) { + int key = cast(rnd.nextInt(size)); + if (rnd.nextInt(50) == 0) { + key = 0; + } + long value = vcast(rnd.nextInt()); + + boolean hadOldValue = map.containsKey(key); + if (rnd.nextBoolean()) { + long previousValue; + if (rnd.nextBoolean()) { + int index = map.indexOf(key); + if (map.indexExists(index)) { + previousValue = map.indexReplace(index, value); + } else { + map.indexInsert(index, key, value); + previousValue = 0L; + } + } else { + previousValue = map.put(key, value); + } + assertEquals( + other.put(key, value), ((previousValue) == 0) && !hadOldValue ? null : previousValue); + + assertEquals(value, map.get(key)); + assertEquals(value, map.indexGet(map.indexOf(key))); + assertTrue(map.containsKey(key)); + assertTrue(map.indexExists(map.indexOf(key))); + } else { + assertEquals(other.containsKey(key), map.containsKey(key)); + long previousValue = + map.containsKey(key) && rnd.nextBoolean() + ? map.indexRemove(map.indexOf(key)) + : map.remove(key); + assertEquals( + other.remove(key), ((previousValue) == 0) && !hadOldValue ? null : previousValue); + } + + assertEquals(other.size(), map.size()); + } + } + } + + /* + * + */ + @Test + public void testClone() { + this.map.put(key1, value1); + this.map.put(key2, value2); + this.map.put(key3, value3); + + IntLongHashMap cloned = map.clone(); + cloned.remove(key1); + + assertSortedListEquals(map.keys().toArray(), key1, key2, key3); + assertSortedListEquals(cloned.keys().toArray(), key2, key3); + } + + /* */ + @Test + public void testMapValues() { + map.put(key1, value3); + map.put(key2, value2); + map.put(key3, value1); + assertSortedListEquals(map.values().toArray(), value1, value2, value3); + + map.clear(); + map.put(key1, value1); + map.put(key2, value2); + map.put(key3, value2); + assertSortedListEquals(map.values().toArray(), value1, value2, value2); + } + + /* */ + @Test + public void testMapValuesIterator() { + map.put(key1, value3); + map.put(key2, value2); + map.put(key3, value1); + + int counted = 0; + for (LongCursor c : map.values()) { + assertEquals(map.values[c.index], c.value); + counted++; + } + assertEquals(counted, map.size()); + } + + /* */ + @Test + public void testEqualsSameClass() { + IntLongHashMap l1 = newInstance(); + l1.put(k1, value0); + l1.put(k2, value1); + l1.put(k3, value2); + + IntLongHashMap l2 = new IntLongHashMap(l1); + l2.putAll(l1); + + IntLongHashMap l3 = new IntLongHashMap(l2); + l3.putAll(l2); + l3.put(k4, value0); + + assertEquals(l1, l2); + assertEquals(l1.hashCode(), l2.hashCode()); + assertNotEquals(l1, l3); + } + + /* */ + @Test + public void testEqualsSubClass() { + class Sub extends IntLongHashMap {} + ; + + IntLongHashMap l1 = newInstance(); + l1.put(k1, value0); + l1.put(k2, value1); + l1.put(k3, value2); + + IntLongHashMap l2 = new Sub(); + l2.putAll(l1); + l2.put(k4, value3); + + IntLongHashMap l3 = new Sub(); + l3.putAll(l2); + + assertNotEquals(l1, l2); + assertEquals(l2.hashCode(), l3.hashCode()); + assertEquals(l2, l3); + } +} diff --git a/lucene/demo/src/java/module-info.java b/lucene/demo/src/java/module-info.java index 13549a56385..9dfc3dcb72d 100644 --- a/lucene/demo/src/java/module-info.java +++ b/lucene/demo/src/java/module-info.java @@ -23,6 +23,7 @@ module org.apache.lucene.demo { requires org.apache.lucene.queries; requires org.apache.lucene.queryparser; requires org.apache.lucene.expressions; + requires org.apache.lucene.sandbox; exports org.apache.lucene.demo; exports org.apache.lucene.demo.facet; diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/SandboxFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/SandboxFacetsExample.java new file mode 100644 index 00000000000..b6759da115b --- /dev/null +++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/SandboxFacetsExample.java @@ -0,0 +1,737 @@ +/* + * 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.demo.facet; + +import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME; +import static org.apache.lucene.sandbox.facet.ComparableUtils.byAggregatedValue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.analysis.core.WhitespaceAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoubleDocValuesField; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.facet.DrillDownQuery; +import org.apache.lucene.facet.DrillSideways; +import org.apache.lucene.facet.FacetField; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.FacetsConfig; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexWriterConfig.OpenMode; +import org.apache.lucene.sandbox.facet.ComparableUtils; +import org.apache.lucene.sandbox.facet.FacetFieldCollector; +import org.apache.lucene.sandbox.facet.FacetFieldCollectorManager; +import org.apache.lucene.sandbox.facet.cutters.TaxonomyFacetsCutter; +import org.apache.lucene.sandbox.facet.cutters.ranges.LongRangeFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.ComparableSupplier; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TaxonomyChildrenOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TopnOrdinalIterator; +import org.apache.lucene.sandbox.facet.labels.RangeOrdToLabel; +import org.apache.lucene.sandbox.facet.labels.TaxonomyOrdLabelBiMap; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.MultiFacetsRecorder; +import org.apache.lucene.sandbox.facet.recorders.Reducer; +import org.apache.lucene.search.CollectorOwner; +import org.apache.lucene.search.DoubleValuesSource; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LongValuesSource; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MultiCollectorManager; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopScoreDocCollectorManager; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; + +/** Demo for sandbox faceting. */ +public class SandboxFacetsExample { + + private final Directory indexDir = new ByteBuffersDirectory(); + private final Directory taxoDir = new ByteBuffersDirectory(); + private final FacetsConfig config = new FacetsConfig(); + + private SandboxFacetsExample() { + config.setHierarchical("Publish Date", true); + } + + /** Build the example index. */ + void index() throws IOException { + IndexWriter indexWriter = + new IndexWriter( + indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE)); + + // Writes facet ords to a separate directory from the main index + DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir); + + Document doc = new Document(); + doc.add(new FacetField("Author", "Bob")); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + doc.add(new NumericDocValuesField("Price", 10)); + doc.add(new NumericDocValuesField("Units", 9)); + doc.add(new DoubleDocValuesField("Popularity", 3.5d)); + indexWriter.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2010", "10", "20")); + doc.add(new NumericDocValuesField("Price", 4)); + doc.add(new NumericDocValuesField("Units", 2)); + doc.add(new DoubleDocValuesField("Popularity", 4.1D)); + indexWriter.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2012", "1", "1")); + doc.add(new NumericDocValuesField("Price", 3)); + doc.add(new NumericDocValuesField("Units", 5)); + doc.add(new DoubleDocValuesField("Popularity", 3.9D)); + indexWriter.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Susan")); + doc.add(new FacetField("Publish Date", "2012", "1", "7")); + doc.add(new NumericDocValuesField("Price", 8)); + doc.add(new NumericDocValuesField("Units", 7)); + doc.add(new DoubleDocValuesField("Popularity", 4D)); + indexWriter.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Frank")); + doc.add(new FacetField("Publish Date", "1999", "5", "5")); + doc.add(new NumericDocValuesField("Price", 9)); + doc.add(new NumericDocValuesField("Units", 6)); + doc.add(new DoubleDocValuesField("Popularity", 4.9D)); + indexWriter.addDocument(config.build(taxoWriter, doc)); + + IOUtils.close(indexWriter, taxoWriter); + } + + /** User runs a query and counts facets only without collecting the matching documents. */ + List facetsOnly() throws IOException { + //// (1) init readers and searcher + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir); + + //// (2) init collector + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + CountFacetRecorder defaultRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, defaultRecorder); + + //// (2.1) if we need to collect data using multiple different collectors, e.g. taxonomy and + //// ranges, or even two taxonomy facets that use different Category List Field, we can + //// use MultiCollectorManager, e.g.: + // TODO: add a demo for it. + // TaxonomyFacetsCutter publishDateCutter = new + // TaxonomyFacetsCutter(config.getDimConfig("Publish Date"), taxoReader); + // CountFacetRecorder publishDateRecorder = new CountFacetRecorder(false); + // FacetFieldCollectorManager publishDateCollectorManager = new + // FacetFieldCollectorManager<>(publishDateCutter, publishDateRecorder); + // MultiCollectorManager drillDownCollectorManager = new + // MultiCollectorManager(authorCollectorManager, publishDateCollectorManager); + // Object[] results = searcher.search(new MatchAllDocsQuery(), drillDownCollectorManager); + + //// (3) search + // Search returns the same Recorder we created - so we can ignore results + searcher.search(new MatchAllDocsQuery(), collectorManager); + + //// (4) Get top 10 results by count for Author and Publish Date + // This object is used to get topN results by count + ComparableSupplier countComparable = + ComparableUtils.byCount(defaultRecorder); + // We don't actually need to use FacetResult, it is up to client what to do with the results. + // Here we just want to demo that we can still do FacetResult as well + List results = new ArrayList<>(2); + // This object provides labels for ordinals. + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + for (String dimension : List.of("Author", "Publish Date")) { + //// (4.1) Chain two ordinal iterators to get top N children + int dimOrdinal = ordLabels.getOrd(new FacetLabel(dimension)); + OrdinalIterator childrenIterator = + new TaxonomyChildrenOrdinalIterator( + defaultRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + dimOrdinal); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIterator, countComparable, 10); + // Get array of final ordinals - we need to use all of them to get labels first, and then to + // get counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = topByCountOrds.toArray(); + + //// (4.2) Use faceting results + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), defaultRecorder.getCount(resultOrdinals[i]))); + } + int dimensionValue = defaultRecorder.getCount(dimOrdinal); + results.add( + new FacetResult( + dimension, + new String[0], + dimensionValue, + labelsAndValues.toArray(new LabelAndValue[0]), + labelsAndValues.size())); + } + + IOUtils.close(indexReader, taxoReader); + return results; + } + + /** + * User runs a query and counts facets for exclusive ranges without collecting the matching + * documents + */ + List exclusiveRangesCountFacetsOnly() throws IOException { + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("Price"); + + // Exclusive ranges example + LongRange[] inputRanges = new LongRange[2]; + inputRanges[0] = new LongRange("0-5", 0, true, 5, true); + inputRanges[1] = new LongRange("5-10", 5, false, 10, true); + + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + searcher.search(new MatchAllDocsQuery(), collectorManager); + RangeOrdToLabel ordToLabels = new RangeOrdToLabel(inputRanges); + + ComparableSupplier countComparable = + ComparableUtils.byCount(countRecorder); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(countRecorder.recordedOrds(), countComparable, 10); + + List results = new ArrayList<>(2); + + int[] resultOrdinals = topByCountOrds.toArray(); + FacetLabel[] labels = ordToLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue(labels[i].lastComponent(), countRecorder.getCount(resultOrdinals[i]))); + } + + results.add( + new FacetResult( + "Price", new String[0], 0, labelsAndValues.toArray(new LabelAndValue[0]), 0)); + + System.out.println("Computed counts"); + IOUtils.close(indexReader); + return results; + } + + List overlappingRangesCountFacetsOnly() throws IOException { + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("Price"); + + // overlapping ranges example + LongRange[] inputRanges = new LongRange[2]; + inputRanges[0] = new LongRange("0-5", 0, true, 5, true); + inputRanges[1] = new LongRange("0-10", 0, true, 10, true); + + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + searcher.search(new MatchAllDocsQuery(), collectorManager); + RangeOrdToLabel ordToLabels = new RangeOrdToLabel(inputRanges); + + ComparableSupplier countComparable = + ComparableUtils.byCount(countRecorder); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(countRecorder.recordedOrds(), countComparable, 10); + + List results = new ArrayList<>(2); + + int[] resultOrdinals = topByCountOrds.toArray(); + FacetLabel[] labels = ordToLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue(labels[i].lastComponent(), countRecorder.getCount(resultOrdinals[i]))); + } + + results.add( + new FacetResult( + "Price", new String[0], 0, labelsAndValues.toArray(new LabelAndValue[0]), 0)); + + System.out.println("Computed counts"); + IOUtils.close(indexReader); + return results; + } + + List exclusiveRangesAggregationFacets() throws IOException { + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("Price"); + + // Exclusive ranges example + LongRange[] inputRanges = new LongRange[2]; + inputRanges[0] = new LongRange("0-5", 0, true, 5, true); + inputRanges[1] = new LongRange("5-10", 5, false, 10, true); + + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + + // initialise the aggregations to be computed - a values source + reducer + LongValuesSource[] longValuesSources = new LongValuesSource[2]; + Reducer[] reducers = new Reducer[2]; + // popularity:max + longValuesSources[0] = DoubleValuesSource.fromDoubleField("Popularity").toLongValuesSource(); + reducers[0] = Reducer.MAX; + // units:sum + longValuesSources[1] = LongValuesSource.fromLongField("Units"); + reducers[1] = Reducer.SUM; + + LongAggregationsFacetRecorder longAggregationsFacetRecorder = + new LongAggregationsFacetRecorder(longValuesSources, reducers); + + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + // Compute both counts and aggregations + MultiFacetsRecorder multiFacetsRecorder = + new MultiFacetsRecorder(countRecorder, longAggregationsFacetRecorder); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, multiFacetsRecorder); + searcher.search(new MatchAllDocsQuery(), collectorManager); + RangeOrdToLabel ordToLabels = new RangeOrdToLabel(inputRanges); + + // Get recorded ords - use either count/aggregations recorder + OrdinalIterator recordedOrds = longAggregationsFacetRecorder.recordedOrds(); + + // We don't actually need to use FacetResult, it is up to client what to do with the results. + // Here we just want to demo that we can still do FacetResult as well + List results = new ArrayList<>(2); + ComparableSupplier comparableSupplier; + OrdinalIterator topOrds; + int[] resultOrdinals; + FacetLabel[] labels; + List labelsAndValues; + + // Sort results by units:sum and tie-break by count + comparableSupplier = byAggregatedValue(countRecorder, longAggregationsFacetRecorder, 1); + topOrds = new TopnOrdinalIterator<>(recordedOrds, comparableSupplier, 10); + + resultOrdinals = topOrds.toArray(); + labels = ordToLabels.getLabels(resultOrdinals); + labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), + longAggregationsFacetRecorder.getRecordedValue(resultOrdinals[i], 1))); + } + results.add( + new FacetResult( + "Price", new String[0], 0, labelsAndValues.toArray(new LabelAndValue[0]), 0)); + + // note: previous ordinal iterator was exhausted + recordedOrds = longAggregationsFacetRecorder.recordedOrds(); + // Sort results by popularity:max and tie-break by count + comparableSupplier = byAggregatedValue(countRecorder, longAggregationsFacetRecorder, 0); + topOrds = new TopnOrdinalIterator<>(recordedOrds, comparableSupplier, 10); + resultOrdinals = topOrds.toArray(); + labels = ordToLabels.getLabels(resultOrdinals); + labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), + longAggregationsFacetRecorder.getRecordedValue(resultOrdinals[i], 0))); + } + results.add( + new FacetResult( + "Price", new String[0], 0, labelsAndValues.toArray(new LabelAndValue[0]), 0)); + + return results; + } + + /** User runs a query and counts facets. */ + private List facetsWithSearch() throws IOException { + //// (1) init readers and searcher + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir); + + //// (2) init collectors + // Facet collectors + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + CountFacetRecorder defaultRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager taxoFacetsCollectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, defaultRecorder); + // Hits collector + TopScoreDocCollectorManager hitsCollectorManager = + new TopScoreDocCollectorManager(2, Integer.MAX_VALUE); + // Now wrap them with MultiCollectorManager to collect both hits and facets. + MultiCollectorManager collectorManager = + new MultiCollectorManager(hitsCollectorManager, taxoFacetsCollectorManager); + + //// (3) search + Object[] results = searcher.search(new MatchAllDocsQuery(), collectorManager); + TopDocs topDocs = (TopDocs) results[0]; + System.out.println( + "Search results: totalHits: " + + topDocs.totalHits + + ", collected hits: " + + topDocs.scoreDocs.length); + // FacetFieldCollectorManager returns the same Recorder it gets - so we can ignore read the + // results from original recorder + // and ignore this value. + // CountFacetRecorder defaultRecorder = (CountFacetRecorder) results[1]; + + //// (4) Get top 10 results by count for Author and Publish Date + // This object is used to get topN results by count + ComparableSupplier countComparable = + ComparableUtils.byCount(defaultRecorder); + // We don't actually need to use FacetResult, it is up to client what to do with the results. + // Here we just want to demo that we can still do FacetResult as well + List facetResults = new ArrayList<>(2); + // This object provides labels for ordinals. + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + for (String dimension : List.of("Author", "Publish Date")) { + int dimensionOrdinal = ordLabels.getOrd(new FacetLabel(dimension)); + //// (4.1) Chain two ordinal iterators to get top N children + OrdinalIterator childrenIterator = + new TaxonomyChildrenOrdinalIterator( + defaultRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + dimensionOrdinal); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIterator, countComparable, 10); + // Get array of final ordinals - we need to use all of them to get labels first, and then to + // get counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = topByCountOrds.toArray(); + + //// (4.2) Use faceting results + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), defaultRecorder.getCount(resultOrdinals[i]))); + } + int dimensionValue = defaultRecorder.getCount(dimensionOrdinal); + facetResults.add( + new FacetResult( + dimension, + new String[0], + dimensionValue, + labelsAndValues.toArray(new LabelAndValue[0]), + labelsAndValues.size())); + } + + IOUtils.close(indexReader, taxoReader); + return facetResults; + } + + /** User drills down on 'Publish Date/2010', and we return facets for 'Author' */ + FacetResult drillDown() throws IOException { + //// (1) init readers and searcher + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir); + + //// (2) init collector + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + CountFacetRecorder defaultRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, defaultRecorder); + + DrillDownQuery q = new DrillDownQuery(config); + q.add("Publish Date", "2010"); + + //// (3) search + // Right now we return the same Recorder we created - so we can ignore results + searcher.search(q, collectorManager); + + //// (4) Get top 10 results by count for Author and Publish Date + // This object is used to get topN results by count + ComparableSupplier countComparable = + ComparableUtils.byCount(defaultRecorder); + + // This object provides labels for ordinals. + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + String dimension = "Author"; + //// (4.1) Chain two ordinal iterators to get top N children + int dimOrdinal = ordLabels.getOrd(new FacetLabel(dimension)); + OrdinalIterator childrenIterator = + new TaxonomyChildrenOrdinalIterator( + defaultRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + dimOrdinal); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIterator, countComparable, 10); + // Get array of final ordinals - we need to use all of them to get labels first, and then to get + // counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = topByCountOrds.toArray(); + + //// (4.2) Use faceting results + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), defaultRecorder.getCount(resultOrdinals[i]))); + } + + IOUtils.close(indexReader, taxoReader); + int dimensionValue = defaultRecorder.getCount(dimOrdinal); + // We don't actually need to use FacetResult, it is up to client what to do with the results. + // Here we just want to demo that we can still do FacetResult as well + return new FacetResult( + dimension, + new String[0], + dimensionValue, + labelsAndValues.toArray(new LabelAndValue[0]), + labelsAndValues.size()); + } + + /** + * User drills down on 'Publish Date/2010', and we return facets for both 'Publish Date' and + * 'Author', using DrillSideways. + */ + private List drillSideways() throws IOException { + //// (1) init readers and searcher + DirectoryReader indexReader = DirectoryReader.open(indexDir); + IndexSearcher searcher = new IndexSearcher(indexReader); + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir); + + //// (2) init drill down query and collectors + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + CountFacetRecorder drillDownRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager drillDownCollectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, drillDownRecorder); + + DrillDownQuery q = new DrillDownQuery(config); + + //// (2.1) add query and collector dimensions + q.add("Publish Date", "2010"); + CountFacetRecorder publishDayDimensionRecorder = new CountFacetRecorder(); + // Note that it is safe to use the same FacetsCutter here because we create Leaf cutter for each + // leaf for each + // FacetFieldCollectorManager anyway, and leaf cutter are not merged or anything like that. + FacetFieldCollectorManager publishDayDimensionCollectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, publishDayDimensionRecorder); + List> drillSidewaysOwners = + List.of(new CollectorOwner<>(publishDayDimensionCollectorManager)); + + //// (3) search + // Right now we return the same Recorder we created - so we can ignore results + DrillSideways ds = new DrillSideways(searcher, config, taxoReader); + // We must wrap list of drill sideways owner with unmodifiableList to make generics work. + ds.search( + q, + new CollectorOwner<>(drillDownCollectorManager), + Collections.unmodifiableList(drillSidewaysOwners)); + + //// (4) Get top 10 results by count for Author + List facetResults = new ArrayList<>(2); + // This object provides labels for ordinals. + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + // This object is used to get topN results by count + ComparableSupplier countComparable = + ComparableUtils.byCount(drillDownRecorder); + //// (4.1) Chain two ordinal iterators to get top N children + int dimOrdinal = ordLabels.getOrd(new FacetLabel("Author")); + OrdinalIterator childrenIterator = + new TaxonomyChildrenOrdinalIterator( + drillDownRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + dimOrdinal); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIterator, countComparable, 10); + // Get array of final ordinals - we need to use all of them to get labels first, and then to get + // counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = topByCountOrds.toArray(); + + //// (4.2) Use faceting results + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), drillDownRecorder.getCount(resultOrdinals[i]))); + } + int dimensionValue = drillDownRecorder.getCount(dimOrdinal); + facetResults.add( + new FacetResult( + "Author", + new String[0], + dimensionValue, + labelsAndValues.toArray(new LabelAndValue[0]), + labelsAndValues.size())); + + //// (5) Same process, but for Publish Date drill sideways dimension + countComparable = ComparableUtils.byCount(publishDayDimensionRecorder); + //// (4.1) Chain two ordinal iterators to get top N children + dimOrdinal = ordLabels.getOrd(new FacetLabel("Publish Date")); + childrenIterator = + new TaxonomyChildrenOrdinalIterator( + publishDayDimensionRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + dimOrdinal); + topByCountOrds = new TopnOrdinalIterator<>(childrenIterator, countComparable, 10); + // Get array of final ordinals - we need to use all of them to get labels first, and then to get + // counts, + // but OrdinalIterator only allows reading ordinals once. + resultOrdinals = topByCountOrds.toArray(); + + //// (4.2) Use faceting results + labels = ordLabels.getLabels(resultOrdinals); + labelsAndValues = new ArrayList<>(labels.length); + for (int i = 0; i < resultOrdinals.length; i++) { + labelsAndValues.add( + new LabelAndValue( + labels[i].lastComponent(), publishDayDimensionRecorder.getCount(resultOrdinals[i]))); + } + dimensionValue = publishDayDimensionRecorder.getCount(dimOrdinal); + facetResults.add( + new FacetResult( + "Publish Date", + new String[0], + dimensionValue, + labelsAndValues.toArray(new LabelAndValue[0]), + labelsAndValues.size())); + + IOUtils.close(indexReader, taxoReader); + return facetResults; + } + + /** Runs the search example. */ + public List runFacetOnly() throws IOException { + index(); + return facetsOnly(); + } + + /** Runs the search example. */ + public List runSearch() throws IOException { + index(); + return facetsWithSearch(); + } + + /** Runs the drill-down example. */ + public FacetResult runDrillDown() throws IOException { + index(); + return drillDown(); + } + + /** Runs the drill-sideways example. */ + public List runDrillSideways() throws IOException { + index(); + return drillSideways(); + } + + /** Runs the example of non overlapping range facets */ + public List runNonOverlappingRangesCountFacetsOnly() throws IOException { + index(); + return exclusiveRangesCountFacetsOnly(); + } + + /** Runs the example of overlapping range facets */ + public List runOverlappingRangesCountFacetsOnly() throws IOException { + index(); + return overlappingRangesCountFacetsOnly(); + } + + /** Runs the example of collecting long aggregations for non overlapping range facets. */ + public List runNonOverlappingRangesAggregationFacets() throws IOException { + index(); + return exclusiveRangesAggregationFacets(); + } + + /** Runs the search and drill-down examples and prints the results. */ + public static void main(String[] args) throws Exception { + System.out.println("Facet counting example:"); + System.out.println("-----------------------"); + SandboxFacetsExample example = new SandboxFacetsExample(); + List results1 = example.runFacetOnly(); + System.out.println("Author: " + results1.get(0)); + System.out.println("Publish Date: " + results1.get(1)); + + System.out.println("Facet counting example (combined facets and search):"); + System.out.println("-----------------------"); + List results = example.runSearch(); + System.out.println("Author: " + results.get(0)); + System.out.println("Publish Date: " + results.get(1)); + + System.out.println("Facet drill-down example (Publish Date/2010):"); + System.out.println("---------------------------------------------"); + System.out.println("Author: " + example.runDrillDown()); + + System.out.println("Facet drill-sideways example (Publish Date/2010):"); + System.out.println("---------------------------------------------"); + for (FacetResult result : example.runDrillSideways()) { + System.out.println(result); + } + + System.out.println("Facet counting example with exclusive ranges:"); + System.out.println("---------------------------------------------"); + for (FacetResult result : example.runNonOverlappingRangesCountFacetsOnly()) { + System.out.println(result); + } + + System.out.println("Facet counting example with overlapping ranges:"); + System.out.println("---------------------------------------------"); + for (FacetResult result : example.runOverlappingRangesCountFacetsOnly()) { + System.out.println(result); + } + + System.out.println("Facet aggregation example with exclusive ranges:"); + System.out.println("---------------------------------------------"); + for (FacetResult result : example.runNonOverlappingRangesAggregationFacets()) { + System.out.println(result); + } + } +} diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java index 2e126634d11..2db660e1c5b 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java @@ -18,6 +18,7 @@ package org.apache.lucene.facet; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -31,6 +32,7 @@ import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState; import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts; import org.apache.lucene.facet.taxonomy.TaxonomyReader; import org.apache.lucene.search.CollectorManager; +import org.apache.lucene.search.CollectorOwner; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; @@ -300,35 +302,25 @@ public class DrillSideways { } } - private static class CallableCollector implements Callable { - - private final int pos; + private static class CallableCollector implements Callable { private final IndexSearcher searcher; private final Query query; - private final CollectorManager collectorManager; + private final CollectorOwner collectorOwner; private CallableCollector( - int pos, IndexSearcher searcher, Query query, CollectorManager collectorManager) { - this.pos = pos; + IndexSearcher searcher, Query query, CollectorOwner collectorOwner) { this.searcher = searcher; this.query = query; - this.collectorManager = collectorManager; + this.collectorOwner = collectorOwner; } @Override - public CallableResult call() throws Exception { - return new CallableResult(pos, searcher.search(query, collectorManager)); - } - } - - private static class CallableResult { - - private final int pos; - private final Object result; - - private CallableResult(int pos, Object result) { - this.pos = pos; - this.result = result; + public Void call() throws Exception { + searcher.search(query, collectorOwner); + // Call getResult to trigger reduce, we don't need to return results because users can access + // them directly from collectorOwner + collectorOwner.getResult(); + return null; } } @@ -349,16 +341,125 @@ public class DrillSideways { public ConcurrentDrillSidewaysResult search( final DrillDownQuery query, final CollectorManager hitCollectorManager) throws IOException { - if (executor != null) { - return searchConcurrently(query, hitCollectorManager); + // Main query + FacetsCollectorManager drillDownFacetsCollectorManager = + createDrillDownFacetsCollectorManager(); + final CollectorOwner mainCollectorOwner; + if (drillDownFacetsCollectorManager != null) { + // Make sure we populate a facet collector corresponding to the base query if desired: + mainCollectorOwner = + new CollectorOwner<>( + new MultiCollectorManager(drillDownFacetsCollectorManager, hitCollectorManager)); } else { - return searchSequentially(query, hitCollectorManager); + mainCollectorOwner = new CollectorOwner<>(hitCollectorManager); + } + // Drill sideways dimensions + final List> drillSidewaysCollectorOwners; + if (query.getDims().isEmpty() == false) { + drillSidewaysCollectorOwners = new ArrayList<>(query.getDims().size()); + for (int i = 0; i < query.getDims().size(); i++) { + drillSidewaysCollectorOwners.add( + new CollectorOwner<>(createDrillSidewaysFacetsCollectorManager())); + } + } else { + drillSidewaysCollectorOwners = null; + } + // Execute query + if (executor != null) { + searchConcurrently(query, mainCollectorOwner, drillSidewaysCollectorOwners); + } else { + searchSequentially(query, mainCollectorOwner, drillSidewaysCollectorOwners); + } + + // Collect results + final FacetsCollector facetsCollectorResult; + final R hitCollectorResult; + if (drillDownFacetsCollectorManager != null) { + // drill down collected using MultiCollector + // Extract the results: + Object[] drillDownResult = (Object[]) mainCollectorOwner.getResult(); + facetsCollectorResult = (FacetsCollector) drillDownResult[0]; + hitCollectorResult = (R) drillDownResult[1]; + } else { + facetsCollectorResult = null; + hitCollectorResult = (R) mainCollectorOwner.getResult(); + } + + // Getting results for drill sideways dimensions (if any) + final String[] drillSidewaysDims; + final FacetsCollector[] drillSidewaysCollectors; + if (query.getDims().isEmpty() == false) { + drillSidewaysDims = query.getDims().keySet().toArray(new String[0]); + int numDims = query.getDims().size(); + assert drillSidewaysCollectorOwners != null; + assert drillSidewaysCollectorOwners.size() == numDims; + drillSidewaysCollectors = new FacetsCollector[numDims]; + for (int dim = 0; dim < numDims; dim++) { + drillSidewaysCollectors[dim] = + (FacetsCollector) drillSidewaysCollectorOwners.get(dim).getResult(); + } + } else { + drillSidewaysDims = null; + drillSidewaysCollectors = null; + } + + return new ConcurrentDrillSidewaysResult<>( + buildFacetsResult(facetsCollectorResult, drillSidewaysCollectors, drillSidewaysDims), + null, + hitCollectorResult, + facetsCollectorResult, + drillSidewaysCollectors, + drillSidewaysDims); + } + + /** + * Search using DrillDownQuery with custom collectors. This method can be used with any {@link + * CollectorOwner}s. It doesn't return anything because it is expected that you read results from + * provided {@link CollectorOwner}s. + * + *

    To read the results, run {@link CollectorOwner#getResult()} for drill down and all drill + * sideways dimensions. + * + *

    Note: use {@link Collections#unmodifiableList(List)} to wrap {@code + * drillSidewaysCollectorOwners} to convince compiler that it is safe to use List here. + * + *

    Use {@link MultiCollectorManager} wrapped by {@link CollectorOwner} to collect both hits and + * facets for the entire query and/or for drill-sideways dimensions. + * + *

    TODO: Class CollectorOwner was created so that we can ignore CollectorManager type C, + * because we want each dimensions to be able to use their own types. Alternatively, we can use + * typesafe heterogeneous container and provide CollectorManager type for each dimension to this + * method? I do like CollectorOwner approach as it seems more intuitive? + */ + public void search( + final DrillDownQuery query, + CollectorOwner drillDownCollectorOwner, + List> drillSidewaysCollectorOwners) + throws IOException { + if (drillDownCollectorOwner == null) { + throw new IllegalArgumentException( + "This search method requires client to provide drill down collector manager"); + } + if (drillSidewaysCollectorOwners == null) { + if (query.getDims().isEmpty() == false) { + throw new IllegalArgumentException( + "The query requires not null drillSidewaysCollectorOwners"); + } + } else if (drillSidewaysCollectorOwners.size() != query.getDims().size()) { + throw new IllegalArgumentException( + "drillSidewaysCollectorOwners size must be equal to number of dimensions in the query."); + } + if (executor != null) { + searchConcurrently(query, drillDownCollectorOwner, drillSidewaysCollectorOwners); + } else { + searchSequentially(query, drillDownCollectorOwner, drillSidewaysCollectorOwners); } } - @SuppressWarnings("unchecked") - private ConcurrentDrillSidewaysResult searchSequentially( - final DrillDownQuery query, final CollectorManager hitCollectorManager) + private void searchSequentially( + final DrillDownQuery query, + final CollectorOwner drillDownCollectorOwner, + final List> drillSidewaysCollectorOwners) throws IOException { Map drillDownDims = query.getDims(); @@ -366,28 +467,9 @@ public class DrillSideways { if (drillDownDims.isEmpty()) { // There are no drill-down dims, so there is no // drill-sideways to compute: - FacetsCollectorManager drillDownCollectorManager = createDrillDownFacetsCollectorManager(); - FacetsCollector mainFacetsCollector; - R collectorResult; - if (drillDownCollectorManager != null) { - Object[] mainResults = - searcher.search( - query, new MultiCollectorManager(drillDownCollectorManager, hitCollectorManager)); - // Extract the results: - mainFacetsCollector = (FacetsCollector) mainResults[0]; - collectorResult = (R) mainResults[1]; - } else { - mainFacetsCollector = null; - collectorResult = searcher.search(query, hitCollectorManager); - } - - return new ConcurrentDrillSidewaysResult<>( - buildFacetsResult(mainFacetsCollector, null, null), - null, - collectorResult, - mainFacetsCollector, - null, - null); + searcher.search(query, drillDownCollectorOwner); + drillDownCollectorOwner.getResult(); + return; } Query baseQuery = query.getBaseQuery(); @@ -398,130 +480,64 @@ public class DrillSideways { } Query[] drillDownQueries = query.getDrillDownQueries(); - int numDims = drillDownDims.size(); - - FacetsCollectorManager drillDownCollectorManager = createDrillDownFacetsCollectorManager(); - - FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers = - new FacetsCollectorManager[numDims]; - for (int i = 0; i < numDims; i++) { - drillSidewaysFacetsCollectorManagers[i] = createDrillSidewaysFacetsCollectorManager(); - } - DrillSidewaysQuery dsq = new DrillSidewaysQuery( baseQuery, - drillDownCollectorManager, - drillSidewaysFacetsCollectorManagers, + // drillDownCollectorOwner, + // Don't pass drill down collector because drill down is collected by IndexSearcher + // itself. + // TODO: deprecate drillDown collection in DrillSidewaysQuery? + null, + drillSidewaysCollectorOwners, drillDownQueries, scoreSubDocsAtOnce()); - R collectorResult = searcher.search(dsq, hitCollectorManager); - - FacetsCollector drillDownCollector; - if (drillDownCollectorManager != null) { - drillDownCollector = drillDownCollectorManager.reduce(dsq.managedDrillDownCollectors); - } else { - drillDownCollector = null; - } - - FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[numDims]; - int numSlices = dsq.managedDrillSidewaysCollectors.size(); - - for (int dim = 0; dim < numDims; dim++) { - List facetsCollectorsForDim = new ArrayList<>(numSlices); - - for (int slice = 0; slice < numSlices; slice++) { - facetsCollectorsForDim.add(dsq.managedDrillSidewaysCollectors.get(slice)[dim]); + searcher.search(dsq, drillDownCollectorOwner); + // This method doesn't return results as each dimension might have its own result type. + // But we call getResult to trigger results reducing, so that users don't have to worry about + // it. + drillDownCollectorOwner.getResult(); + if (drillSidewaysCollectorOwners != null) { + for (CollectorOwner sidewaysOwner : drillSidewaysCollectorOwners) { + sidewaysOwner.getResult(); } - - drillSidewaysCollectors[dim] = - drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim); } - - String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]); - - return new ConcurrentDrillSidewaysResult<>( - buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillSidewaysDims), - null, - collectorResult, - drillDownCollector, - drillSidewaysCollectors, - drillSidewaysDims); } - @SuppressWarnings("unchecked") - private ConcurrentDrillSidewaysResult searchConcurrently( - final DrillDownQuery query, final CollectorManager hitCollectorManager) + private void searchConcurrently( + final DrillDownQuery query, + final CollectorOwner drillDownCollectorOwner, + final List> drillSidewaysCollectorOwners) throws IOException { final Map drillDownDims = query.getDims(); final List callableCollectors = new ArrayList<>(drillDownDims.size() + 1); - // Add the main DrillDownQuery - FacetsCollectorManager drillDownFacetsCollectorManager = - createDrillDownFacetsCollectorManager(); - CollectorManager mainCollectorManager; - if (drillDownFacetsCollectorManager != null) { - // Make sure we populate a facet collector corresponding to the base query if desired: - mainCollectorManager = - new MultiCollectorManager(drillDownFacetsCollectorManager, hitCollectorManager); - } else { - mainCollectorManager = hitCollectorManager; - } - callableCollectors.add(new CallableCollector(-1, searcher, query, mainCollectorManager)); + callableCollectors.add(new CallableCollector(searcher, query, drillDownCollectorOwner)); int i = 0; final Query[] filters = query.getDrillDownQueries(); - for (String dim : drillDownDims.keySet()) + for (String dim : drillDownDims.keySet()) { callableCollectors.add( new CallableCollector( - i++, searcher, getDrillDownQuery(query, filters, dim), - createDrillSidewaysFacetsCollectorManager())); - - final FacetsCollector mainFacetsCollector; - final FacetsCollector[] facetsCollectors = new FacetsCollector[drillDownDims.size()]; - final R collectorResult; + drillSidewaysCollectorOwners.get(i))); + i++; + } try { // Run the query pool - final List> futures = executor.invokeAll(callableCollectors); + final List> futures = executor.invokeAll(callableCollectors); - // Extract the results - if (drillDownFacetsCollectorManager != null) { - // If we populated a facets collector for the main query, make sure to unpack it properly - final Object[] mainResults = (Object[]) futures.get(0).get().result; - mainFacetsCollector = (FacetsCollector) mainResults[0]; - collectorResult = (R) mainResults[1]; - } else { - mainFacetsCollector = null; - collectorResult = (R) futures.get(0).get().result; + // Wait for results. We don't read the results as they are collected by CollectorOwners + for (i = 0; i < futures.size(); i++) { + futures.get(i).get(); } - for (i = 1; i < futures.size(); i++) { - final CallableResult result = futures.get(i).get(); - facetsCollectors[result.pos] = (FacetsCollector) result.result; - } - // Fill the null results with the mainFacetsCollector - for (i = 0; i < facetsCollectors.length; i++) - if (facetsCollectors[i] == null) facetsCollectors[i] = mainFacetsCollector; - } catch (InterruptedException e) { throw new ThreadInterruptedException(e); } catch (ExecutionException e) { throw new RuntimeException(e); } - - String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]); - - // build the facets and return the result - return new ConcurrentDrillSidewaysResult<>( - buildFacetsResult(mainFacetsCollector, facetsCollectors, drillSidewaysDims), - null, - collectorResult, - mainFacetsCollector, - facetsCollectors, - drillSidewaysDims); } /** diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java index dca425f40f4..aa670914d1a 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java @@ -17,14 +17,14 @@ package org.apache.lucene.facet; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.CollectorOwner; import org.apache.lucene.search.ConstantScoreScorer; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Explanation; @@ -45,10 +45,8 @@ class DrillSidewaysQuery extends Query { final Query baseQuery; - final FacetsCollectorManager drillDownCollectorManager; - final FacetsCollectorManager[] drillSidewaysCollectorManagers; - final List managedDrillDownCollectors; - final List managedDrillSidewaysCollectors; + final CollectorOwner drillDownCollectorOwner; + final List> drillSidewaysCollectorOwners; final Query[] drillDownQueries; @@ -56,47 +54,17 @@ class DrillSidewaysQuery extends Query { /** * Construct a new {@code DrillSidewaysQuery} that will create new {@link FacetsCollector}s for - * each {@link LeafReaderContext} using the provided {@link FacetsCollectorManager}s. The caller - * can access the created {@link FacetsCollector}s through {@link #managedDrillDownCollectors} and - * {@link #managedDrillSidewaysCollectors}. + * each {@link LeafReaderContext} using the provided {@link FacetsCollectorManager}s. */ DrillSidewaysQuery( Query baseQuery, - FacetsCollectorManager drillDownCollectorManager, - FacetsCollectorManager[] drillSidewaysCollectorManagers, - Query[] drillDownQueries, - boolean scoreSubDocsAtOnce) { - // Note that the "managed" facet collector lists are synchronized here since bulkScorer() - // can be invoked concurrently and needs to remain thread-safe. We're OK with synchronizing - // on the whole list as contention is expected to remain very low: - this( - baseQuery, - drillDownCollectorManager, - drillSidewaysCollectorManagers, - Collections.synchronizedList(new ArrayList<>()), - Collections.synchronizedList(new ArrayList<>()), - drillDownQueries, - scoreSubDocsAtOnce); - } - - /** - * Needed for {@link Query#rewrite(IndexSearcher)}. Ensures the same "managed" lists get used - * since {@link DrillSideways} accesses references to these through the original {@code - * DrillSidewaysQuery}. - */ - private DrillSidewaysQuery( - Query baseQuery, - FacetsCollectorManager drillDownCollectorManager, - FacetsCollectorManager[] drillSidewaysCollectorManagers, - List managedDrillDownCollectors, - List managedDrillSidewaysCollectors, + CollectorOwner drillDownCollectorOwner, + List> drillSidewaysCollectorOwners, Query[] drillDownQueries, boolean scoreSubDocsAtOnce) { this.baseQuery = Objects.requireNonNull(baseQuery); - this.drillDownCollectorManager = drillDownCollectorManager; - this.drillSidewaysCollectorManagers = drillSidewaysCollectorManagers; - this.managedDrillDownCollectors = managedDrillDownCollectors; - this.managedDrillSidewaysCollectors = managedDrillSidewaysCollectors; + this.drillDownCollectorOwner = drillDownCollectorOwner; + this.drillSidewaysCollectorOwners = drillSidewaysCollectorOwners; this.drillDownQueries = drillDownQueries; this.scoreSubDocsAtOnce = scoreSubDocsAtOnce; } @@ -121,10 +89,8 @@ class DrillSidewaysQuery extends Query { } else { return new DrillSidewaysQuery( newQuery, - drillDownCollectorManager, - drillSidewaysCollectorManagers, - managedDrillDownCollectors, - managedDrillSidewaysCollectors, + drillDownCollectorOwner, + drillSidewaysCollectorOwners, drillDownQueries, scoreSubDocsAtOnce); } @@ -158,20 +124,15 @@ class DrillSidewaysQuery extends Query { int drillDownCount = drillDowns.length; - FacetsCollector drillDownCollector; - LeafCollector drillDownLeafCollector; - if (drillDownCollectorManager != null) { - drillDownCollector = drillDownCollectorManager.newCollector(); - managedDrillDownCollectors.add(drillDownCollector); + Collector drillDownCollector; + final LeafCollector drillDownLeafCollector; + if (drillDownCollectorOwner != null) { + drillDownCollector = drillDownCollectorOwner.newCollector(); drillDownLeafCollector = drillDownCollector.getLeafCollector(context); } else { - drillDownCollector = null; drillDownLeafCollector = null; } - FacetsCollector[] sidewaysCollectors = new FacetsCollector[drillDownCount]; - managedDrillSidewaysCollectors.add(sidewaysCollectors); - DrillSidewaysScorer.DocsAndCost[] dims = new DrillSidewaysScorer.DocsAndCost[drillDownCount]; @@ -183,8 +144,7 @@ class DrillSidewaysQuery extends Query { scorer = new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty()); } - FacetsCollector sidewaysCollector = drillSidewaysCollectorManagers[dim].newCollector(); - sidewaysCollectors[dim] = sidewaysCollector; + Collector sidewaysCollector = drillSidewaysCollectorOwners.get(dim).newCollector(); dims[dim] = new DrillSidewaysScorer.DocsAndCost( @@ -195,11 +155,11 @@ class DrillSidewaysQuery extends Query { // a null scorer in this case, but we need to make sure #finish gets called on all facet // collectors since IndexSearcher won't handle this for us: if (baseScorerSupplier == null || nullCount > 1) { - if (drillDownCollector != null) { - drillDownCollector.finish(); + if (drillDownLeafCollector != null) { + drillDownLeafCollector.finish(); } - for (FacetsCollector fc : sidewaysCollectors) { - fc.finish(); + for (DrillSidewaysScorer.DocsAndCost dim : dims) { + dim.sidewaysLeafCollector.finish(); } return null; } @@ -252,9 +212,9 @@ class DrillSidewaysQuery extends Query { final int prime = 31; int result = classHash(); result = prime * result + Objects.hashCode(baseQuery); - result = prime * result + Objects.hashCode(drillDownCollectorManager); + result = prime * result + Objects.hashCode(drillDownCollectorOwner); result = prime * result + Arrays.hashCode(drillDownQueries); - result = prime * result + Arrays.hashCode(drillSidewaysCollectorManagers); + result = prime * result + Objects.hashCode(drillSidewaysCollectorOwners); return result; } @@ -265,8 +225,8 @@ class DrillSidewaysQuery extends Query { private boolean equalsTo(DrillSidewaysQuery other) { return Objects.equals(baseQuery, other.baseQuery) - && Objects.equals(drillDownCollectorManager, other.drillDownCollectorManager) + && Objects.equals(drillDownCollectorOwner, other.drillDownCollectorOwner) && Arrays.equals(drillDownQueries, other.drillDownQueries) - && Arrays.equals(drillSidewaysCollectorManagers, other.drillSidewaysCollectorManagers); + && Objects.equals(drillSidewaysCollectorOwners, other.drillSidewaysCollectorOwners); } } diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiDoubleValuesSource.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiDoubleValuesSource.java index 44427965b8c..9a53df274c3 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/MultiDoubleValuesSource.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiDoubleValuesSource.java @@ -26,6 +26,7 @@ import org.apache.lucene.search.DoubleValues; import org.apache.lucene.search.DoubleValuesSource; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SegmentCacheable; +import org.apache.lucene.util.NumericUtils; /** * Base class for producing {@link MultiDoubleValues}. See also {@link DoubleValuesSource} for a @@ -118,6 +119,65 @@ public abstract class MultiDoubleValuesSource implements SegmentCacheable { return new LongDoubleValuesSource(this); } + /** Convert to a {@link MultiLongValuesSource} using {@link NumericUtils#doubleToSortableLong} */ + public final MultiLongValuesSource toSortableMultiLongValuesSource() { + return new SortableMultiLongValuesSource(this); + } + + private static class SortableMultiLongValuesSource extends MultiLongValuesSource { + + MultiDoubleValuesSource inner; + + SortableMultiLongValuesSource(MultiDoubleValuesSource inner) { + this.inner = Objects.requireNonNull(inner); + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return inner.isCacheable(ctx); + } + + @Override + public MultiLongValues getValues(LeafReaderContext ctx) throws IOException { + MultiDoubleValues doubleValues = inner.getValues(ctx); + + return new MultiLongValues() { + @Override + public long getValueCount() { + return doubleValues.getValueCount(); + } + + @Override + public long nextValue() throws IOException { + return NumericUtils.doubleToSortableLong(doubleValues.nextValue()); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + return doubleValues.advanceExact(doc); + } + }; + } + + @Override + public int hashCode() { + return inner.hashCode(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SortableMultiLongValuesSource that = (SortableMultiLongValuesSource) o; + return Objects.equals(inner, that.inner); + } + + @Override + public String toString() { + return "sortableMultiLong(" + inner.toString() + ")"; + } + } + private static class FieldMultiValuedSource extends MultiDoubleValuesSource { private final String field; private final LongToDoubleFunction decoder; diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java index 4696e16e4ec..8c8181f0706 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java @@ -178,6 +178,14 @@ public class FacetLabel implements Comparable { } } + /** Get the last component. */ + public String lastComponent() { + if (components.length == 0) { + throw new UnsupportedOperationException("components is empty"); + } + return components[components.length - 1]; + } + /** Returns a string representation of the path. */ @Override public String toString() { diff --git a/lucene/sandbox/build.gradle b/lucene/sandbox/build.gradle index 93f01e3f96d..72762fe1c3d 100644 --- a/lucene/sandbox/build.gradle +++ b/lucene/sandbox/build.gradle @@ -22,5 +22,6 @@ description = 'Various third party contributions and new ideas' dependencies { moduleApi project(':lucene:core') moduleApi project(':lucene:queries') + moduleApi project(':lucene:facet') moduleTestImplementation project(':lucene:test-framework') } diff --git a/lucene/sandbox/src/java/module-info.java b/lucene/sandbox/src/java/module-info.java index 3daace50cee..f40a05af433 100644 --- a/lucene/sandbox/src/java/module-info.java +++ b/lucene/sandbox/src/java/module-info.java @@ -19,6 +19,7 @@ module org.apache.lucene.sandbox { requires org.apache.lucene.core; requires org.apache.lucene.queries; + requires org.apache.lucene.facet; exports org.apache.lucene.payloads; exports org.apache.lucene.sandbox.codecs.idversion; @@ -27,6 +28,12 @@ module org.apache.lucene.sandbox { exports org.apache.lucene.sandbox.queries; exports org.apache.lucene.sandbox.search; exports org.apache.lucene.sandbox.index; + exports org.apache.lucene.sandbox.facet; + exports org.apache.lucene.sandbox.facet.recorders; + exports org.apache.lucene.sandbox.facet.cutters.ranges; + exports org.apache.lucene.sandbox.facet.iterators; + exports org.apache.lucene.sandbox.facet.cutters; + exports org.apache.lucene.sandbox.facet.labels; provides org.apache.lucene.codecs.PostingsFormat with org.apache.lucene.sandbox.codecs.idversion.IDVersionPostingsFormat; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java new file mode 100644 index 00000000000..aa6e8fd0775 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java @@ -0,0 +1,261 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import org.apache.lucene.sandbox.facet.cutters.LongValueFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.ComparableSupplier; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder; +import org.apache.lucene.util.InPlaceMergeSorter; + +/** + * Collection of static methods to provide most common comparables for sandbox faceting. You can + * also use it as an example for creating your own {@link ComparableSupplier} to enable custom + * facets top-n and sorting. + * + * @lucene.experimental + */ +public final class ComparableUtils { + private ComparableUtils() {} + + /** {@link ComparableSupplier} to sort by ords (ascending). */ + public static ComparableSupplier byOrdinal() { + return new ComparableSupplier<>() { + public void reuseComparable(int ord, ByOrdinalComparable reuse) { + reuse.ord = ord; + } + + public ByOrdinalComparable createComparable(int ord) { + ByOrdinalComparable result = new ByOrdinalComparable(); + result.ord = ord; + return result; + } + }; + } + + /** Used for {@link #byOrdinal} result. */ + public static class ByOrdinalComparable implements Comparable { + + private int ord; + + @Override + public int compareTo(ByOrdinalComparable o) { + return Integer.compare(o.ord, ord); + } + } + + /** + * {@link ComparableSupplier} to sort ordinals by count (descending) with ord as a tie-break + * (ascending) using provided {@link CountFacetRecorder}. + */ + public static ComparableSupplier byCount(CountFacetRecorder recorder) { + return new ComparableSupplier<>() { + public void reuseComparable(int ord, ByCountComparable reuse) { + reuse.ord = ord; + reuse.count = recorder.getCount(ord); + } + + public ByCountComparable createComparable(int ord) { + ByCountComparable result = new ByCountComparable(); + result.ord = ord; + result.count = recorder.getCount(ord); + return result; + } + }; + } + + /** Used for {@link #byCount} result. */ + public static class ByCountComparable implements Comparable { + private ByCountComparable() {} + + private int count; + private int ord; + + @Override + public int compareTo(ByCountComparable o) { + int cmp = Integer.compare(count, o.count); + if (cmp == 0) { + cmp = Integer.compare(o.ord, ord); + } + return cmp; + } + } + + /** + * {@link ComparableSupplier} to sort ordinals by long aggregation (descending) with tie-break by + * count (descending) or by ordinal (ascending) using provided {@link CountFacetRecorder} and + * {@link LongAggregationsFacetRecorder}. + */ + public static ComparableSupplier byAggregatedValue( + CountFacetRecorder countRecorder, + LongAggregationsFacetRecorder longAggregationsFacetRecorder, + int aggregationId) { + return new ComparableSupplier<>() { + public void reuseComparable(int ord, ByAggregatedValueComparable reuse) { + reuse.ord = ord; + reuse.secondaryRank = countRecorder.getCount(ord); + reuse.primaryRank = longAggregationsFacetRecorder.getRecordedValue(ord, aggregationId); + } + + public ByAggregatedValueComparable createComparable(int ord) { + ByAggregatedValueComparable result = new ByAggregatedValueComparable(); + reuseComparable(ord, result); + return result; + } + }; + } + + /** Used for {@link #byAggregatedValue} result. */ + public static class ByAggregatedValueComparable + implements Comparable { + private ByAggregatedValueComparable() {} + + private int ord; + private int secondaryRank; + private long primaryRank; + + @Override + public int compareTo(ByAggregatedValueComparable o) { + int cmp = Long.compare(primaryRank, o.primaryRank); + if (cmp == 0) { + cmp = Integer.compare(secondaryRank, o.secondaryRank); + if (cmp == 0) { + cmp = Integer.compare(o.ord, ord); + } + } + return cmp; + } + } + + /** + * {@link ComparableSupplier} to sort ordinals by long value from {@link LongValueFacetCutter} + * (descending). + */ + public static ComparableSupplier byLongValue( + LongValueFacetCutter longValueFacetCutter) { + return new ComparableSupplier<>() { + public void reuseComparable(int ord, ByLongValueComparable reuse) { + reuse.value = longValueFacetCutter.getValue(ord); + } + + public ByLongValueComparable createComparable(int ord) { + ByLongValueComparable result = new ByLongValueComparable(); + result.value = longValueFacetCutter.getValue(ord); + return result; + } + }; + } + + /** Used for {@link #byLongValue} result. */ + public static final class ByLongValueComparable implements Comparable { + private ByLongValueComparable() {} + + private long value; + + @Override + public int compareTo(ByLongValueComparable o) { + return Long.compare(o.value, value); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ByLongValueComparable other) { + return other.value == value; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(value); + } + } + + /** + * {@link ComparableSupplier} to sort ordinals by count (descending) from {@link + * CountFacetRecorder} with tie-break by long value (ascending) from {@link LongValueFacetCutter}. + */ + public static ComparableSupplier byCount( + CountFacetRecorder countFacetRecorder, LongValueFacetCutter longValueFacetCutter) { + return new ComparableSupplier<>() { + public void reuseComparable(int ord, ByCountAndLongValueComparable reuse) { + reuse.value = longValueFacetCutter.getValue(ord); + reuse.count = countFacetRecorder.getCount(ord); + } + + public ByCountAndLongValueComparable createComparable(int ord) { + ByCountAndLongValueComparable result = new ByCountAndLongValueComparable(); + reuseComparable(ord, result); + return result; + } + }; + } + + /** Used for {@link #byCount(CountFacetRecorder, LongValueFacetCutter)} result. */ + public static class ByCountAndLongValueComparable + implements Comparable { + private ByCountAndLongValueComparable() {} + + private int count; + private long value; + + @Override + public int compareTo(ByCountAndLongValueComparable o) { + int cmp = Integer.compare(count, o.count); + if (cmp == 0) { + cmp = Long.compare(o.value, value); + } + return cmp; + } + } + + /** + * Sort array of ordinals. + * + *

    To get top-n ordinals use {@link + * org.apache.lucene.sandbox.facet.iterators.TopnOrdinalIterator} instead. + * + * @param ordinals array of ordinals to sort + * @param comparableSupplier defines sort order + */ + public static > void sort( + int[] ordinals, ComparableSupplier comparableSupplier) throws IOException { + List comparables = new ArrayList<>(ordinals.length); + for (int i = 0; i < ordinals.length; i++) { + comparables.add(comparableSupplier.createComparable(ordinals[i])); + } + new InPlaceMergeSorter() { + @Override + protected void swap(int i, int j) { + int tmp = ordinals[i]; + ordinals[i] = ordinals[j]; + ordinals[j] = tmp; + Collections.swap(comparables, i, j); + } + + @Override + protected int compare(int i, int j) { + return comparables.get(j).compareTo(comparables.get(i)); + } + }.sort(0, ordinals.length); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollector.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollector.java new file mode 100644 index 00000000000..90158bd0650 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollector.java @@ -0,0 +1,59 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.ScoreMode; + +/** + * {@link Collector} that brings together {@link FacetCutter} and {@link FacetRecorder} to compute + * facets during collection phase. + * + * @lucene.experimental + */ +public final class FacetFieldCollector implements Collector { + private final FacetCutter facetCutter; + private final FacetRecorder facetRecorder; + + /** Collector for cutter+recorder pair. */ + public FacetFieldCollector(FacetCutter facetCutter, FacetRecorder facetRecorder) { + this.facetCutter = facetCutter; + this.facetRecorder = facetRecorder; + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return new FacetFieldLeafCollector(context, facetCutter, facetRecorder); + } + + @Override + public ScoreMode scoreMode() { + // TODO: Some FacetRecorders might need scores, e.g. to get associated numeric values, see for + // example TaxonomyFacetFloatAssociations. Not sure if anyone actually uses it, because + // FacetsCollectorManager creates FacetsCollector with keepScores: false. But if someone needs + // it, we can add boolean needScores method to FacetRecorder interface, return + // ScoreMode.COMPLETE here when the method returns true. FacetRecorders#needScores should be + // implemented on case by case basis, e.g. LongAggregationsFacetRecorder can take it as a + // constuctor argument, and when it's true call LongValues#getValues with the scores. + return ScoreMode.COMPLETE_NO_SCORES; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollectorManager.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollectorManager.java new file mode 100644 index 00000000000..f78aebcff24 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollectorManager.java @@ -0,0 +1,53 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import java.util.Collection; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; +import org.apache.lucene.search.CollectorManager; + +/** + * Collector manager for {@link FacetFieldCollector}. Returns the same extension of {@link + * FacetRecorder} that was used to collect results. + * + * @lucene.experimental + */ +public final class FacetFieldCollectorManager + implements CollectorManager { + + private final FacetCutter facetCutter; + private final V facetRecorder; + + /** Create collector for a cutter + recorder pair */ + public FacetFieldCollectorManager(FacetCutter facetCutter, V facetRecorder) { + this.facetCutter = facetCutter; + this.facetRecorder = facetRecorder; + } + + @Override + public FacetFieldCollector newCollector() throws IOException { + return new FacetFieldCollector(facetCutter, facetRecorder); + } + + @Override + public V reduce(Collection collectors) throws IOException { + facetRecorder.reduce(facetCutter); + return this.facetRecorder; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldLeafCollector.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldLeafCollector.java new file mode 100644 index 00000000000..3361022be8b --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldLeafCollector.java @@ -0,0 +1,76 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.LeafFacetRecorder; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorable; + +/** + * {@link LeafCollector} that for each facet ordinal from {@link LeafFacetCutter} records data with + * {@link LeafFacetRecorder}. + */ +final class FacetFieldLeafCollector implements LeafCollector { + + private final LeafReaderContext context; + private final FacetCutter cutter; + private final FacetRecorder recorder; + private LeafFacetCutter leafCutter; + + private LeafFacetRecorder leafRecorder; + + FacetFieldLeafCollector(LeafReaderContext context, FacetCutter cutter, FacetRecorder recorder) { + this.context = context; + this.cutter = cutter; + this.recorder = recorder; + } + + @Override + public void setScorer(Scorable scorer) throws IOException { + // TODO: see comment in FacetFieldCollector#scoreMode + } + + @Override + public void collect(int doc) throws IOException { + if (leafCutter == null) { + leafCutter = cutter.createLeafCutter(context); + assert leafRecorder == null; + leafRecorder = recorder.getLeafRecorder(context); + } + if (leafCutter.advanceExact(doc)) { + for (int curOrd = leafCutter.nextOrd(); + curOrd != LeafFacetCutter.NO_MORE_ORDS; + curOrd = leafCutter.nextOrd()) { + leafRecorder.record(doc, curOrd); + } + } + } + + @Override + public DocIdSetIterator competitiveIterator() throws IOException { + // TODO: any ideas? + // 1. Docs that have values for the index field we about to facet on + // 2. TK + return LeafCollector.super.competitiveIterator(); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/FacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/FacetCutter.java new file mode 100644 index 00000000000..7f33af313e0 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/FacetCutter.java @@ -0,0 +1,54 @@ +/* + * 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.sandbox.facet.cutters; + +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * Creates {@link LeafFacetCutter} for each leaf. + * + *

    TODO: do we need FacetCutterManager similar to CollectorManager, e.g. is createLeafCutter + * always thread safe? + * + * @lucene.experimental + */ +public interface FacetCutter { + + /** Get cutter for the leaf. */ + LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException; + + /** + * For facets that have hierarchy (levels), return all top level dimension ordinals that require + * rollup. + * + *

    Rollup is an optimization for facets types that support hierarchy, if single document + * belongs to at most one node in the hierarchy, we can first record data for these nodes only, + * and then roll up values to parent ordinals. + * + *

    Default implementation returns null, which means that rollup is not needed. + */ + default OrdinalIterator getOrdinalsToRollup() throws IOException { + return null; + } + + /** For facets that have hierarchy (levels), get all children ordinals for given ord. */ + default OrdinalIterator getChildrenOrds(int ord) throws IOException { + return null; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LeafFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LeafFacetCutter.java new file mode 100644 index 00000000000..fdfe1b3600c --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LeafFacetCutter.java @@ -0,0 +1,34 @@ +/* + * 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.sandbox.facet.cutters; + +import java.io.IOException; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * Interface to be implemented to cut documents into facets for an index segment (leaf). + * + *

    When {@link #advanceExact(int)} returns true, {@link #nextOrd()} yields all facet ordinals for + * the current document. It is illegal to call {@link #nextOrd()} if {@link #advanceExact(int)} + * returns false. + * + * @lucene.experimental + */ +public interface LeafFacetCutter extends OrdinalIterator { + /** advance to the next doc */ + boolean advanceExact(int doc) throws IOException; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java new file mode 100644 index 00000000000..1ec32c86375 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java @@ -0,0 +1,187 @@ +/* + * 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.sandbox.facet.cutters; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.IntSupplier; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.internal.hppc.IntLongHashMap; +import org.apache.lucene.internal.hppc.LongIntHashMap; +import org.apache.lucene.sandbox.facet.labels.OrdToLabel; + +/** + * {@link FacetCutter} and {@link OrdToLabel} for distinct long values. + * + *

    TODO: This class is quite inefficient. Will optimise later. + * + *

    TODO: add support for other value sources e.g: LongValues + * + * @lucene.experimental + */ +public final class LongValueFacetCutter implements FacetCutter, OrdToLabel { + private final String field; + // TODO: consider alternatives if this is a bottleneck + private final LongIntHashMapSyncCompute valueToOrdMap; + private IntLongHashMap ordToValueMap; + private final AtomicInteger maxOrdinal; + + /** + * Constructor. + * + * @param field field name to read long values from. + */ + public LongValueFacetCutter(String field) { + this.field = field; + valueToOrdMap = new LongIntHashMapSyncCompute(); + ordToValueMap = null; + maxOrdinal = new AtomicInteger(-1); + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), field); + return new LeafFacetCutter() { + int docValueCount; + long lastDocValue; + int docValueCursor; + + @Override + public boolean advanceExact(int doc) throws IOException { + if (docValues.advanceExact(doc)) { + docValueCount = docValues.docValueCount(); + docValueCursor = 0; + return true; + } + return false; + } + + @Override + public int nextOrd() throws IOException { + while (docValueCursor++ < docValueCount) { + long value = docValues.nextValue(); + // SortedNumericDocValues can have duplicates, but values are sorted, so we only need to + // check previous value to remove duplicates + if (docValueCursor == 1 || value != lastDocValue) { + lastDocValue = value; + return valueToOrdMap.computeIfAbsent(value, maxOrdinal::incrementAndGet); + } + } + return NO_MORE_ORDS; + } + }; + } + + @Override + public FacetLabel getLabel(int ordinal) { + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + if (ordToValueMap.containsKey(ordinal)) { + return new FacetLabel(String.valueOf(ordToValueMap.get(ordinal))); + } + assert false + : "ordinal=" + + ordinal + + ", ordToValueMap.size=" + + ordToValueMap.size() + + ", valueToOrdMap.size=" + + valueToOrdMap.size(); + return null; + } + + /** + * Get value by ordinal. Should only be called after collection phase. + * + *

    TODO: we need it to tie break sort by value. Alternatively we can sort by label (then we + * don't need this method), but we would have to convert FacetLabel to "long" to have the same + * order... Overall, it is probably not important to tie break by value, and we can tie break by + * ord same as for other facets; but for now we don't want to change results order just in case. + * + * @param ordinal facet ordinal. + * @return long value + */ + public long getValue(int ordinal) { + // TODO: do we want to create #finish method that called by #reduce to build the map? + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + return ordToValueMap.get(ordinal); + } + + private void buildOrdToValueMap() { + ordToValueMap = new IntLongHashMap(valueToOrdMap.size()); + for (LongIntHashMap.LongIntCursor cursor : valueToOrdMap) { + ordToValueMap.put(cursor.value, cursor.key); + } + } + + @Override + public FacetLabel[] getLabels(int[] ordinals) throws IOException { + FacetLabel[] facetLabels = new FacetLabel[ordinals.length]; + for (int i = 0; i < ordinals.length; i++) { + facetLabels[i] = getLabel(ordinals[i]); + } + return facetLabels; + } + + /** {@link LongIntHashMap} with threadsafe computeIfAbsent method */ + private static class LongIntHashMapSyncCompute extends LongIntHashMap { + private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); + private final Lock r = rwl.readLock(); + private final Lock w = rwl.writeLock(); + + /** + * If key exists in the map return its value, otherwise insert value from the value supplier and + * return it. + * + *

    The method is threadsafe, and it allows concurrent reading from the map, but it locks the + * map to insert a new value as it might require rehashing. + */ + public int computeIfAbsent(long key, IntSupplier valueSupplier) { + r.lock(); + int value; + try { + value = super.getOrDefault(key, -1); + } finally { + r.unlock(); + } + if (value == -1) { + w.lock(); + try { + int index = super.indexOf(key); + if (super.indexExists(index)) { + return super.indexGet(index); + } else { + value = valueSupplier.getAsInt(); + super.indexInsert(index, key, value); + return value; + } + } finally { + w.unlock(); + } + } else { + return value; + } + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/TaxonomyFacetsCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/TaxonomyFacetsCutter.java new file mode 100644 index 00000000000..cbefdd8feeb --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/TaxonomyFacetsCutter.java @@ -0,0 +1,199 @@ +/* + * 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.sandbox.facet.cutters; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Map; +import org.apache.lucene.facet.FacetsConfig; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.ParallelTaxonomyArrays; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * {@link FacetCutter} for facets that use taxonomy side-car index. + * + * @lucene.experimental + */ +public final class TaxonomyFacetsCutter implements FacetCutter { + + private final FacetsConfig facetsConfig; + private final TaxonomyReader taxoReader; + private final String indexFieldName; + private final boolean disableRollup; + + private ParallelTaxonomyArrays.IntArray children; + private ParallelTaxonomyArrays.IntArray siblings; + + /** Create {@link FacetCutter} for taxonomy facets. */ + public TaxonomyFacetsCutter( + String indexFieldName, FacetsConfig facetsConfig, TaxonomyReader taxoReader) { + this(indexFieldName, facetsConfig, taxoReader, false); + } + + /** + * Expert: Create {@link FacetCutter} for taxonomy facets. + * + * @param disableRollup if set to true, rollup is disabled. In most cases users should not use it. + * Setting it to true silently leads to incorrect results for dimensions that require rollup. + * At the same time, if you are sure that there are no dimensions that require rollup, setting + * it to true might improve performance. + */ + public TaxonomyFacetsCutter( + String indexFieldName, + FacetsConfig facetsConfig, + TaxonomyReader taxoReader, + boolean disableRollup) { + this.facetsConfig = facetsConfig; + this.indexFieldName = indexFieldName; + this.taxoReader = taxoReader; + this.disableRollup = disableRollup; + } + + /** + * Returns int[] mapping each ordinal to its first child; this is a large array and is computed + * (and then saved) the first time this method is invoked. + */ + ParallelTaxonomyArrays.IntArray getChildren() throws IOException { + if (children == null) { + children = taxoReader.getParallelTaxonomyArrays().children(); + } + return children; + } + + /** + * Returns int[] mapping each ordinal to its next sibling; this is a large array and is computed + * (and then saved) the first time this method is invoked. + */ + ParallelTaxonomyArrays.IntArray getSiblings() throws IOException { + if (siblings == null) { + siblings = taxoReader.getParallelTaxonomyArrays().siblings(); + } + return siblings; + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + SortedNumericDocValues multiValued = + DocValues.getSortedNumeric(context.reader(), indexFieldName); + // DocValues.getSortedNumeric never returns null + assert multiValued != null; + // TODO: if multiValued is emptySortedNumeric we can throw CollectionTerminatedException + // in FacetFieldLeafCollector and save some CPU cycles. + TaxonomyLeafFacetCutterMultiValue leafCutter = + new TaxonomyLeafFacetCutterMultiValue(multiValued); + return leafCutter; + + // TODO: does unwrapping Single valued make things any faster? We still need to wrap it into + // LeafFacetCutter + // NumericDocValues singleValued = DocValues.unwrapSingleton(multiValued); + } + + @Override + public OrdinalIterator getOrdinalsToRollup() throws IOException { + if (disableRollup) { + return null; + } + + // Rollup any necessary dims: + Iterator> dimensions = + facetsConfig.getDimConfigs().entrySet().iterator(); + + ArrayList dimsToRollup = new ArrayList<>(); + + while (dimensions.hasNext()) { + Map.Entry ent = dimensions.next(); + String dim = ent.getKey(); + FacetsConfig.DimConfig ft = ent.getValue(); + if (ft.hierarchical && ft.multiValued == false && ft.indexFieldName.equals(indexFieldName)) { + dimsToRollup.add(new FacetLabel(dim)); + } + } + + int[] dimOrdToRollup = taxoReader.getBulkOrdinals(dimsToRollup.toArray(new FacetLabel[0])); + + return new OrdinalIterator() { + int currentIndex = 0; + + @Override + public int nextOrd() throws IOException { + for (; currentIndex < dimOrdToRollup.length; currentIndex++) { + // It can be invalid if this field was declared in the + // config but never indexed + if (dimOrdToRollup[currentIndex] != TaxonomyReader.INVALID_ORDINAL) { + return dimOrdToRollup[currentIndex++]; + } + } + return NO_MORE_ORDS; + } + }; + } + + @Override + public OrdinalIterator getChildrenOrds(final int parentOrd) throws IOException { + ParallelTaxonomyArrays.IntArray children = getChildren(); + ParallelTaxonomyArrays.IntArray siblings = getSiblings(); + return new OrdinalIterator() { + int currentChild = parentOrd; + + @Override + public int nextOrd() { + if (currentChild == parentOrd) { + currentChild = children.get(currentChild); + } else { + currentChild = siblings.get(currentChild); + } + if (currentChild != TaxonomyReader.INVALID_ORDINAL) { + return currentChild; + } + return NO_MORE_ORDS; + } + }; + } + + private static class TaxonomyLeafFacetCutterMultiValue implements LeafFacetCutter { + private final SortedNumericDocValues multiValued; + private int ordsInDoc; + + private TaxonomyLeafFacetCutterMultiValue(SortedNumericDocValues multiValued) { + this.multiValued = multiValued; + } + + @Override + public int nextOrd() throws IOException { + if (ordsInDoc > 0) { + ordsInDoc--; + return (int) multiValued.nextValue(); + } + return LeafFacetCutter.NO_MORE_ORDS; + } + + @Override + public boolean advanceExact(int doc) throws IOException { + if (multiValued.advanceExact(doc)) { + ordsInDoc = multiValued.docValueCount(); + return true; + } + return false; + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/package-info.java new file mode 100644 index 00000000000..83e72212fe2 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ +/** + * Sandbox faceting: facet cutters, see {@link org.apache.lucene.sandbox.facet.cutters.FacetCutter} + * for details. + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet.cutters; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/DoubleRangeFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/DoubleRangeFacetCutter.java new file mode 100644 index 00000000000..69c295355a3 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/DoubleRangeFacetCutter.java @@ -0,0 +1,90 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import java.io.IOException; +import org.apache.lucene.facet.MultiDoubleValuesSource; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.DoubleRange; +import org.apache.lucene.facet.range.DoubleRangeFacetCounts; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.search.DoubleValuesSource; +import org.apache.lucene.search.LongValuesSource; +import org.apache.lucene.util.NumericUtils; + +/** + * {@link FacetCutter} for ranges of double values. + * + *

    Based on {@link DoubleRangeFacetCounts}, this class translates double ranges to long ranges + * using {@link NumericUtils#doubleToSortableLong} and delegates faceting work to a {@link + * LongRangeFacetCutter}. + * + * @lucene.experimental + */ +public final class DoubleRangeFacetCutter implements FacetCutter { + + private final LongRangeFacetCutter longRangeFacetCutter; + + /** Constructor. */ + public DoubleRangeFacetCutter( + MultiDoubleValuesSource multiDoubleValuesSource, DoubleRange[] doubleRanges) { + super(); + DoubleValuesSource singleDoubleValuesSource = + MultiDoubleValuesSource.unwrapSingleton(multiDoubleValuesSource); + LongValuesSource singleLongValuesSource; + MultiLongValuesSource multiLongValuesSource; + if (singleDoubleValuesSource != null) { + singleLongValuesSource = singleDoubleValuesSource.toSortableLongDoubleValuesSource(); + multiLongValuesSource = null; + } else { + singleLongValuesSource = null; + multiLongValuesSource = multiDoubleValuesSource.toSortableMultiLongValuesSource(); + } + LongRange[] longRanges = mapDoubleRangesToSortableLong(doubleRanges); + // TODO: instead of relying on either single value source or multi value source to be null, we + // should create different factory methods for single and multi valued versions and use the + // right one + this.longRangeFacetCutter = + LongRangeFacetCutter.createSingleOrMultiValued( + multiLongValuesSource, singleLongValuesSource, longRanges); + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + return longRangeFacetCutter.createLeafCutter(context); + } + + // TODO: it is exactly the same as DoubleRangeFacetCounts#getLongRanges (protected), we should + // dedup + private LongRange[] mapDoubleRangesToSortableLong(DoubleRange[] doubleRanges) { + LongRange[] longRanges = new LongRange[doubleRanges.length]; + for (int i = 0; i < longRanges.length; i++) { + DoubleRange dr = doubleRanges[i]; + longRanges[i] = + new LongRange( + dr.label, + NumericUtils.doubleToSortableLong(dr.min), + true, + NumericUtils.doubleToSortableLong(dr.max), + true); + } + return longRanges; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/IntervalTracker.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/IntervalTracker.java new file mode 100644 index 00000000000..f3b11f56296 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/IntervalTracker.java @@ -0,0 +1,95 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import java.io.IOException; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.util.FixedBitSet; + +/** + * A specialised ordinal iterator that supports write (set and clear) operations. Clients can write + * data and freeze the state before reading data from it like any other OrdinalIterator. Instances + * may be reused by clearing the current iterator E.g. LongRangeFacetCutter uses IntervalTracker + * instances to map ranges to ordinals and track per-range data and retrieve recorded ranges for a + * data set. + * + * @lucene.experimental + */ +interface IntervalTracker extends OrdinalIterator { + /** track information for the seen input interval * */ + void set(int i); + + /** clear recorded information on this tracker. * */ + void clear(); + + /** check if any data for the interval has been recorded * */ + boolean get(int index); + + /** finalise any state before read operations can be performed on this OrdinalIterator */ + void freeze(); + + /** + * Interval Tracker that tracks data for multiple intervals. The interval is recorded only once + * iff data belonging to the interval is encountered * + */ + class MultiIntervalTracker implements IntervalTracker { + + private FixedBitSet tracker; + private int trackerState; + private int bitFrom; + + private int intervalsWithHit; + + MultiIntervalTracker(int size) { + tracker = new FixedBitSet(size); + } + + @Override + public void set(int i) { + tracker.set(i); + } + + @Override + public void clear() { + tracker.clear(); + bitFrom = 0; + trackerState = 0; + intervalsWithHit = 0; + } + + @Override + public boolean get(int index) { + return tracker.get(index); + } + + @Override + public void freeze() { + intervalsWithHit = tracker.cardinality(); + } + + @Override + public int nextOrd() throws IOException { + if (trackerState == intervalsWithHit) { + return NO_MORE_ORDS; + } + trackerState++; + int nextSetBit = tracker.nextSetBit(bitFrom); + bitFrom = nextSetBit + 1; + return nextSetBit; + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java new file mode 100644 index 00000000000..49fd4b1317f --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java @@ -0,0 +1,320 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import org.apache.lucene.facet.MultiLongValues; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.search.LongValues; +import org.apache.lucene.search.LongValuesSource; + +/** + * {@link FacetCutter} for ranges of long values. It's based on LongRangeCounter class. + * + * @lucene.experimental + */ +public abstract class LongRangeFacetCutter implements FacetCutter { + + final MultiLongValuesSource valuesSource; + + // TODO: refactor - weird that we have both multi and single here. + final LongValuesSource singleValues; + final LongRangeAndPos[] sortedRanges; + + final int requestedRangeCount; + + final List elementaryIntervals; + + /** elementary interval boundaries used for efficient counting (bsearch to find interval) */ + final long[] boundaries; + + final int[] pos; + + // Default interval position, when elementary interval is mapped to this interval + // it is skipped. + static final int SKIP_INTERVAL_POSITION = -1; + + /** Create {@link FacetCutter} for provided value source and long ranges. */ + static LongRangeFacetCutter createSingleOrMultiValued( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + if (areOverlappingRanges(longRanges)) { + return new OverlappingLongRangeFacetCutter( + longValuesSource, singleLongValuesSource, longRanges); + } + return new NonOverlappingLongRangeFacetCutter( + longValuesSource, singleLongValuesSource, longRanges); + } + + public static LongRangeFacetCutter create( + MultiLongValuesSource longValuesSource, LongRange[] longRanges) { + return createSingleOrMultiValued(longValuesSource, null, longRanges); + } + + // caller handles conversion of Doubles and DoubleRange to Long and LongRange + // ranges need not be sorted + LongRangeFacetCutter( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + super(); + valuesSource = longValuesSource; + if (singleLongValuesSource != null) { + singleValues = singleLongValuesSource; + } else { + singleValues = MultiLongValuesSource.unwrapSingleton(valuesSource); + } + + sortedRanges = new LongRangeAndPos[longRanges.length]; + requestedRangeCount = longRanges.length; + + for (int i = 0; i < longRanges.length; i++) { + sortedRanges[i] = new LongRangeAndPos(longRanges[i], i); + } + + Arrays.sort(this.sortedRanges, Comparator.comparingLong(r -> r.range.min)); + elementaryIntervals = buildElementaryIntervals(); + + // Keep track of elementary interval boundary ends (for binary search) along with the requested + // range they map back to (and -1 when they map to a "gap" range in case of ExclusiveRanges): + boundaries = new long[elementaryIntervals.size()]; + pos = new int[elementaryIntervals.size()]; + Arrays.fill(pos, SKIP_INTERVAL_POSITION); + int currRange = 0; + for (int i = 0; i < boundaries.length; i++) { + boundaries[i] = elementaryIntervals.get(i).end; + if (currRange < sortedRanges.length) { + LongRangeAndPos curr = sortedRanges[currRange]; + if (boundaries[i] == curr.range.max) { + pos[i] = curr.pos; + currRange++; + } + } + } + } + + /** + * Generates non-overlapping intervals that cover requested ranges and gaps in-between. Each + * elementary range refers to a gap, single requested range, or multiple requested ranges when + * they overlap. + */ + abstract List buildElementaryIntervals(); + + private static boolean areOverlappingRanges(LongRange[] ranges) { + if (ranges.length == 0) { + return false; + } + + // Copy before sorting so we don't mess with the caller's original ranges: + // TODO: We're going to do this again in the constructor. Can't we come up with a clever way to + // avoid doing it twice? + LongRange[] sortedRanges = new LongRange[ranges.length]; + System.arraycopy(ranges, 0, sortedRanges, 0, ranges.length); + Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.min)); + + long previousMax = sortedRanges[0].max; + for (int i = 1; i < sortedRanges.length; i++) { + // Ranges overlap if the next min is <= the previous max (note that LongRange models + // closed ranges, so equal limit points are considered overlapping): + if (sortedRanges[i].min <= previousMax) { + return true; + } + previousMax = sortedRanges[i].max; + } + + return false; + } + + abstract static class LongRangeMultivaluedLeafFacetCutter implements LeafFacetCutter { + private final MultiLongValues multiLongValues; + private final long[] boundaries; + final int[] pos; + final IntervalTracker elementaryIntervalTracker; + + // TODO: we need it only for overlapping ranges, should not handle it in advanceExact for + // exclusive ranges. + IntervalTracker requestedIntervalTracker; + + LongRangeMultivaluedLeafFacetCutter(MultiLongValues longValues, long[] boundaries, int[] pos) { + this.multiLongValues = longValues; + this.boundaries = boundaries; + this.pos = pos; + elementaryIntervalTracker = new IntervalTracker.MultiIntervalTracker(boundaries.length); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + if (multiLongValues.advanceExact(doc) == false) { + return false; + } + + elementaryIntervalTracker.clear(); + + if (requestedIntervalTracker != null) { + requestedIntervalTracker.clear(); + } + + long numValues = multiLongValues.getValueCount(); + + int lastIntervalSeen = -1; + + for (int i = 0; i < numValues; i++) { + lastIntervalSeen = processValue(multiLongValues.nextValue(), lastIntervalSeen); + assert lastIntervalSeen >= 0 && lastIntervalSeen < boundaries.length; + elementaryIntervalTracker.set(lastIntervalSeen); + if (lastIntervalSeen == boundaries.length - 1) { + // we've already reached the end of all possible intervals for this doc + break; + } + } + maybeRollUp(requestedIntervalTracker); + + elementaryIntervalTracker.freeze(); + + if (requestedIntervalTracker != null) { + requestedIntervalTracker.freeze(); + } + + return true; + } + + // Returns the value of the interval v belongs or lastIntervalSeen + // if no processing is done, it returns the lastIntervalSeen + private int processValue(long v, int lastIntervalSeen) { + int lo = 0, hi = boundaries.length - 1; + + if (lastIntervalSeen != -1) { + // this is the multivalued doc case, we need to set lo correctly + if (v <= boundaries[lastIntervalSeen]) { + // we've already counted something for this interval and doc + // we don't need to process v + return lastIntervalSeen; + } + + lo = lastIntervalSeen + 1; + if (lo == boundaries.length) { + // we've already counted the last elementary interval. If so, there's nothing + // else to count for this doc + // TODO: does it make sense to return something else? + return lastIntervalSeen; + } + } + int lowerBound = lo; + + while (true) { + int mid = (lo + hi) >>> 1; + if (v <= boundaries[mid]) { + if (mid == lowerBound) { + return mid; + } else { + hi = mid - 1; + } + } else if (v > boundaries[mid + 1]) { + lo = mid + 1; + } else { + return mid + 1; + } + } + } + + void maybeRollUp(IntervalTracker rollUpInto) {} + } + + abstract static class LongRangeSingleValuedLeafFacetCutter implements LeafFacetCutter { + private final LongValues longValues; + private final long[] boundaries; + final int[] pos; + int elementaryIntervalOrd; + + IntervalTracker requestedIntervalTracker; + + LongRangeSingleValuedLeafFacetCutter(LongValues longValues, long[] boundaries, int[] pos) { + this.longValues = longValues; + this.boundaries = boundaries; + this.pos = pos; + } + + @Override + public boolean advanceExact(int doc) throws IOException { + if (longValues.advanceExact(doc) == false) { + return false; + } + if (requestedIntervalTracker != null) { + requestedIntervalTracker.clear(); + } + elementaryIntervalOrd = processValue(longValues.longValue()); + maybeRollUp(requestedIntervalTracker); + if (requestedIntervalTracker != null) { + requestedIntervalTracker.freeze(); + } + + return true; + } + + // Returns the value of the interval v belongs or lastIntervalSeen + // if no processing is done, it returns the lastIntervalSeen + private int processValue(long v) { + int lo = 0, hi = boundaries.length - 1; + + int lowerBound = lo; + + while (true) { + int mid = (lo + hi) >>> 1; + if (v <= boundaries[mid]) { + if (mid == lowerBound) { + return mid; + } else { + hi = mid - 1; + } + } else if (v > boundaries[mid + 1]) { + lo = mid + 1; + } else { + return mid + 1; + } + } + } + + void maybeRollUp(IntervalTracker rollUpInto) {} + } + + record LongRangeAndPos(LongRange range, int pos) { + @Override + public String toString() { + return "LongRangeAndPos[" + "range=" + range + ", " + "pos=" + pos + ']'; + } + } + + /** + * Similar to InclusiveRange from LongRangeCounter. + * + *

    TODO: dedup + */ + record InclusiveRange(long start, long end) { + + @Override + public String toString() { + return start + " to " + end; + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeNode.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeNode.java new file mode 100644 index 00000000000..576d6ecf465 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeNode.java @@ -0,0 +1,94 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import org.apache.lucene.internal.hppc.IntArrayList; + +/** + * Holds one node of the segment tree. + * + *

    TODO: dedup existing LongRangeNode. + */ +final class LongRangeNode { + final LongRangeNode left; + final LongRangeNode right; + + // Our range, inclusive: + final long start; + final long end; + + // Which range indices to output when a query goes + // through this node: + IntArrayList outputs; + + /** add doc * */ + LongRangeNode(long start, long end, LongRangeNode left, LongRangeNode right) { + this.start = start; + this.end = end; + this.left = left; + this.right = right; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + toString(sb, 0); + return sb.toString(); + } + + private static void indent(StringBuilder sb, int depth) { + sb.append(" ".repeat(depth)); + } + + /** Recursively assigns range outputs to each node. */ + public void addOutputs(LongRangeFacetCutter.LongRangeAndPos range) { + if (start >= range.range().min && end <= range.range().max) { + // Our range is fully included in the incoming + // range; add to our output list: + if (outputs == null) { + outputs = new IntArrayList(); + } + outputs.add(range.pos()); + } else if (left != null) { + assert right != null; + // Recurse: + left.addOutputs(range); + right.addOutputs(range); + } + } + + private void toString(StringBuilder sb, int depth) { + indent(sb, depth); + if (left == null) { + assert right == null; + sb.append("leaf: ").append(start).append(" to ").append(end); + } else { + sb.append("node: ").append(start).append(" to ").append(end); + } + if (outputs != null) { + sb.append(" outputs="); + sb.append(outputs); + } + sb.append('\n'); + + if (left != null) { + assert right != null; + left.toString(sb, depth + 1); + right.toString(sb, depth + 1); + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/NonOverlappingLongRangeFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/NonOverlappingLongRangeFacetCutter.java new file mode 100644 index 00000000000..3d657a96570 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/NonOverlappingLongRangeFacetCutter.java @@ -0,0 +1,125 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.lucene.facet.MultiLongValues; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.search.LongValues; +import org.apache.lucene.search.LongValuesSource; + +/** {@link LongRangeFacetCutter} for ranges of long value that don't overlap. * */ +class NonOverlappingLongRangeFacetCutter extends LongRangeFacetCutter { + NonOverlappingLongRangeFacetCutter( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + super(longValuesSource, singleLongValuesSource, longRanges); + } + + /** + * TODO: it's identical to private ExclusiveLongRangeCounter#buildElementaryIntervals, let's + * dedup. + */ + @Override + List buildElementaryIntervals() { + List elementaryIntervals = new ArrayList<>(); + long prev = Long.MIN_VALUE; + for (LongRangeAndPos range : sortedRanges) { + if (range.range().min > prev) { + // add a "gap" range preceding requested range if necessary: + elementaryIntervals.add(new InclusiveRange(prev, range.range().min - 1)); + } + // add the requested range: + elementaryIntervals.add(new InclusiveRange(range.range().min, range.range().max)); + prev = range.range().max + 1; + } + if (elementaryIntervals.isEmpty() == false) { + long lastEnd = elementaryIntervals.get(elementaryIntervals.size() - 1).end(); + if (lastEnd < Long.MAX_VALUE) { + elementaryIntervals.add(new InclusiveRange(lastEnd + 1, Long.MAX_VALUE)); + } + } else { + // If no ranges were requested, create a single entry from MIN_VALUE to MAX_VALUE: + elementaryIntervals.add(new InclusiveRange(Long.MIN_VALUE, Long.MAX_VALUE)); + } + + return elementaryIntervals; + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + if (singleValues != null) { + LongValues values = singleValues.getValues(context, null); + return new NonOverlappingLongRangeSingleValueLeafFacetCutter(values, boundaries, pos); + } else { + MultiLongValues values = valuesSource.getValues(context); + return new NonOverlappingLongRangeMultiValueLeafFacetCutter(values, boundaries, pos); + } + } + + /** + * TODO: dedup NonOverlappingLongRangeMultiValueLeafFacetCutter and + * NonOverlappingLongRangeSingleValueLeafFacetCutter code - they are similar but they extend + * different base classes. + */ + static class NonOverlappingLongRangeMultiValueLeafFacetCutter + extends LongRangeMultivaluedLeafFacetCutter { + + NonOverlappingLongRangeMultiValueLeafFacetCutter( + MultiLongValues longValues, long[] boundaries, int[] pos) { + super(longValues, boundaries, pos); + } + + @Override + public int nextOrd() throws IOException { + while (true) { + int ordinal = elementaryIntervalTracker.nextOrd(); + if (ordinal == NO_MORE_ORDS) { + return NO_MORE_ORDS; + } + int result = pos[ordinal]; + if (result != SKIP_INTERVAL_POSITION) { + return result; + } + } + } + } + + static class NonOverlappingLongRangeSingleValueLeafFacetCutter + extends LongRangeSingleValuedLeafFacetCutter { + NonOverlappingLongRangeSingleValueLeafFacetCutter( + LongValues longValues, long[] boundaries, int[] pos) { + super(longValues, boundaries, pos); + } + + @Override + public int nextOrd() throws IOException { + if (elementaryIntervalOrd == NO_MORE_ORDS) { + return NO_MORE_ORDS; + } + int result = pos[elementaryIntervalOrd]; + elementaryIntervalOrd = NO_MORE_ORDS; + return result != SKIP_INTERVAL_POSITION ? result : NO_MORE_ORDS; + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/OverlappingLongRangeFacetCutter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/OverlappingLongRangeFacetCutter.java new file mode 100644 index 00000000000..58586db892f --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/OverlappingLongRangeFacetCutter.java @@ -0,0 +1,273 @@ +/* + * 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.sandbox.facet.cutters.ranges; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.facet.MultiLongValues; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.internal.hppc.IntCursor; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.search.LongValues; +import org.apache.lucene.search.LongValuesSource; + +/** + * {@link LongRangeFacetCutter} for ranges of long value that overlap. Uses segment tree + * optimisation to find all matching ranges for a given value fast-range-faceting- + * using-segment-trees.html + */ +class OverlappingLongRangeFacetCutter extends LongRangeFacetCutter { + private final LongRangeNode root; + + OverlappingLongRangeFacetCutter( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + super(longValuesSource, singleLongValuesSource, longRanges); + + // Build binary tree on top of intervals: + root = split(0, elementaryIntervals.size(), elementaryIntervals); + + // Set outputs, so we know which range to output for each node in the tree: + for (LongRangeAndPos range : sortedRanges) { + root.addOutputs(range); + } + } + + /** + * TODO: it's identical to private OverlappingLongRangeCounter#buildElementaryIntervals, let's + * dedup. + */ + @Override + List buildElementaryIntervals() { + // Maps all range inclusive endpoints to int flags; 1 + // = start of interval, 2 = end of interval. We need to + // track the start vs end case separately because if a + // given point is both, then it must be its own + // elementary interval: + Map endsMap = new HashMap<>(); + + endsMap.put(Long.MIN_VALUE, 1); + endsMap.put(Long.MAX_VALUE, 2); + + for (LongRangeAndPos rangeAndPos : sortedRanges) { + Integer cur = endsMap.get(rangeAndPos.range().min); + if (cur == null) { + endsMap.put(rangeAndPos.range().min, 1); + } else { + endsMap.put(rangeAndPos.range().min, cur | 1); + } + cur = endsMap.get(rangeAndPos.range().max); + if (cur == null) { + endsMap.put(rangeAndPos.range().max, 2); + } else { + endsMap.put(rangeAndPos.range().max, cur | 2); + } + } + + List endsList = new ArrayList<>(endsMap.keySet()); + Collections.sort(endsList); + + // Build elementaryIntervals (a 1D Venn diagram): + List elementaryIntervals = new ArrayList<>(); + int upto = 1; + long v = endsList.get(0); + long prev; + if (endsMap.get(v) == 3) { + elementaryIntervals.add(new InclusiveRange(v, v)); + prev = v + 1; + } else { + prev = v; + } + + while (upto < endsList.size()) { + v = endsList.get(upto); + int flags = endsMap.get(v); + if (flags == 3) { + // This point is both an end and a start; we need to + // separate it: + if (v > prev) { + elementaryIntervals.add(new InclusiveRange(prev, v - 1)); + } + elementaryIntervals.add(new InclusiveRange(v, v)); + prev = v + 1; + } else if (flags == 1) { + // This point is only the start of an interval; + // attach it to next interval: + if (v > prev) { + elementaryIntervals.add(new InclusiveRange(prev, v - 1)); + } + prev = v; + } else { + assert flags == 2; + // This point is only the end of an interval; attach + // it to last interval: + elementaryIntervals.add(new InclusiveRange(prev, v)); + prev = v + 1; + } + upto++; + } + + return elementaryIntervals; + } + + private static LongRangeNode split(int start, int end, List elementaryIntervals) { + if (start == end - 1) { + // leaf + InclusiveRange range = elementaryIntervals.get(start); + return new LongRangeNode(range.start(), range.end(), null, null); + } else { + int mid = (start + end) >>> 1; + LongRangeNode left = split(start, mid, elementaryIntervals); + LongRangeNode right = split(mid, end, elementaryIntervals); + return new LongRangeNode(left.start, right.end, left, right); + } + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + if (singleValues != null) { + LongValues values = singleValues.getValues(context, null); + return new OverlappingSingleValuedRangeLeafFacetCutter( + values, boundaries, pos, requestedRangeCount, root); + } else { + MultiLongValues values = valuesSource.getValues(context); + return new OverlappingMultivaluedRangeLeafFacetCutter( + values, boundaries, pos, requestedRangeCount, root); + } + } + + /** + * TODO: dedup OverlappingMultivaluedRangeLeafFacetCutter and + * OverlappingSingleValuedRangeLeafFacetCutter code - they are identical but they extend different + * base classes. + */ + static class OverlappingMultivaluedRangeLeafFacetCutter + extends LongRangeMultivaluedLeafFacetCutter { + + private final LongRangeNode elementaryIntervalRoot; + + private int elementaryIntervalUpto; + + OverlappingMultivaluedRangeLeafFacetCutter( + MultiLongValues longValues, + long[] boundaries, + int[] pos, + int requestedRangeCount, + LongRangeNode elementaryIntervalRoot) { + super(longValues, boundaries, pos); + requestedIntervalTracker = new IntervalTracker.MultiIntervalTracker(requestedRangeCount); + this.elementaryIntervalRoot = elementaryIntervalRoot; + } + + @Override + void maybeRollUp(IntervalTracker rollUpInto) { + elementaryIntervalUpto = 0; + rollupMultiValued(elementaryIntervalRoot); + } + + private boolean rollupMultiValued(LongRangeNode node) { + boolean containedHit; + if (node.left != null) { + containedHit = rollupMultiValued(node.left); + containedHit |= rollupMultiValued(node.right); + } else { + // Leaf: + containedHit = elementaryIntervalTracker.get(elementaryIntervalUpto); + elementaryIntervalUpto++; + } + if (containedHit && node.outputs != null) { + for (IntCursor rangeIndex : node.outputs) { + requestedIntervalTracker.set(rangeIndex.value); + } + } + + return containedHit; + } + + @Override + public int nextOrd() throws IOException { + if (requestedIntervalTracker == null) { + return NO_MORE_ORDS; + } + return requestedIntervalTracker.nextOrd(); + } + } + + static class OverlappingSingleValuedRangeLeafFacetCutter + extends LongRangeSingleValuedLeafFacetCutter { + + private final LongRangeNode elementaryIntervalRoot; + + private int elementaryIntervalUpto; + + OverlappingSingleValuedRangeLeafFacetCutter( + LongValues longValues, + long[] boundaries, + int[] pos, + int requestedRangeCount, + LongRangeNode elementaryIntervalRoot) { + super(longValues, boundaries, pos); + requestedIntervalTracker = new IntervalTracker.MultiIntervalTracker(requestedRangeCount); + this.elementaryIntervalRoot = elementaryIntervalRoot; + } + + @Override + void maybeRollUp(IntervalTracker rollUpInto) { + // TODO: for single valued we can rollup after collecting all documents, e.g. in reduce + // method. Maybe we can use FacetCutter rollup methods to handle this case too? + elementaryIntervalUpto = 0; + rollupSingleValued(elementaryIntervalRoot); + } + + // Note: combined rollUpSingleValued and rollUpMultiValued from OverlappingLongRangeCounter into + // 1 rollUp method + private boolean rollupSingleValued(LongRangeNode node) { + boolean containedHit; + if (node.left != null) { + containedHit = rollupSingleValued(node.left); + containedHit |= rollupSingleValued(node.right); + } else { + // Leaf: + containedHit = elementaryIntervalUpto == elementaryIntervalOrd; + elementaryIntervalUpto++; + } + if (containedHit && node.outputs != null) { + for (IntCursor rangeIndex : node.outputs) { + requestedIntervalTracker.set(rangeIndex.value); + } + } + + return containedHit; + } + + @Override + public int nextOrd() throws IOException { + if (requestedIntervalTracker == null) { + return NO_MORE_ORDS; + } + return requestedIntervalTracker.nextOrd(); + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/package-info.java new file mode 100644 index 00000000000..7d76f6218f4 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Sandbox faceting: Range Faceting + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet.cutters.ranges; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/CandidateSetOrdinalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/CandidateSetOrdinalIterator.java new file mode 100644 index 00000000000..503530513c7 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/CandidateSetOrdinalIterator.java @@ -0,0 +1,63 @@ +/* + * 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.sandbox.facet.iterators; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.sandbox.facet.labels.LabelToOrd; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; + +/** + * {@link OrdinalIterator} that filters out ordinals from delegate if they are not in the candidate + * set. Can be handy to get results only for specific facets. + * + * @lucene.experimental + */ +public final class CandidateSetOrdinalIterator implements OrdinalIterator { + + private final OrdinalIterator candidateOrdinalIterator; + private final FacetRecorder facetRecorder; + + /** Constructor. */ + public CandidateSetOrdinalIterator( + FacetRecorder facetRecorder, FacetLabel[] candidateLabels, LabelToOrd labelToOrd) + throws IOException { + // TODO: if candidates size >> number of ordinals in facetRecorder, it is more efficient to + // iterate ordinals from FacetRecorder, and check if candidates contain them + if (facetRecorder.isEmpty()) { + // Getting ordinals for labels might be expensive, e.g. it requires reading index for taxonomy + // facets, so we make sure we don't do it for empty facet recorder. + this.candidateOrdinalIterator = OrdinalIterator.EMPTY; + } else { + this.candidateOrdinalIterator = + OrdinalIterator.fromArray(labelToOrd.getOrds(candidateLabels)); + } + this.facetRecorder = facetRecorder; + } + + @Override + public int nextOrd() throws IOException { + for (int ord = candidateOrdinalIterator.nextOrd(); + ord != NO_MORE_ORDS; + ord = candidateOrdinalIterator.nextOrd()) { + if (facetRecorder.contains(ord)) { + return ord; + } + } + return NO_MORE_ORDS; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/ComparableSupplier.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/ComparableSupplier.java new file mode 100644 index 00000000000..d31e2575770 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/ComparableSupplier.java @@ -0,0 +1,43 @@ +/* + * 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.sandbox.facet.iterators; + +/** + * Generates {@link Comparable} for provided ordinal. For example, it can be used to find topN facet + * ordinals. + * + * @param something ordinals can be compared by. + * @lucene.experimental + */ +public interface ComparableSupplier> { + + /** + * For given ordinal, get something it can be compared by. + * + * @param ord ordinal. + * @param reuse object to reuse for building result. Must not be null. + */ + void reuseComparable(int ord, T reuse); + + /** + * For given ordinal, create something it can be compared by. + * + * @param ord ordinal. + * @return Comparable. + */ + T createComparable(int ord); +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/OrdinalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/OrdinalIterator.java new file mode 100644 index 00000000000..c5e0bfc2822 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/OrdinalIterator.java @@ -0,0 +1,69 @@ +/* + * 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.sandbox.facet.iterators; + +import java.io.IOException; +import org.apache.lucene.internal.hppc.IntArrayList; + +/** + * Iterate over ordinals. + * + * @lucene.experimental + */ +public interface OrdinalIterator { + + /** This const is returned by nextOrd when there are no more ordinals. */ + int NO_MORE_ORDS = -1; + + /** Returns next ord or {@link #NO_MORE_ORDS}. * */ + int nextOrd() throws IOException; + + /** + * Convert to int array. Note that after this method is called original OrdinalIterator is + * exhausted. + */ + default int[] toArray() throws IOException { + IntArrayList resultList = new IntArrayList(); + for (int ord = this.nextOrd(); ord != NO_MORE_ORDS; ord = this.nextOrd()) { + resultList.add(ord); + } + return resultList.toArray(); + } + + /** Convert int array to ordinal iterator. */ + static OrdinalIterator fromArray(int[] source) { + return new OrdinalIterator() { + int cursor; + + @Override + public int nextOrd() throws IOException { + int ord; + while (cursor < source.length) { + ord = source[cursor++]; + // NO_MORE_ORDS should be returned only after we read the entire array. + if (ord != NO_MORE_ORDS) { + return ord; + } + } + return NO_MORE_ORDS; + } + }; + } + + /** Return empty ordinal iterator */ + OrdinalIterator EMPTY = () -> NO_MORE_ORDS; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TaxonomyChildrenOrdinalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TaxonomyChildrenOrdinalIterator.java new file mode 100644 index 00000000000..84ab5875467 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TaxonomyChildrenOrdinalIterator.java @@ -0,0 +1,58 @@ +/* + * 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.sandbox.facet.iterators; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.ParallelTaxonomyArrays; +import org.apache.lucene.sandbox.facet.labels.LabelToOrd; + +/** + * Facets results selector to get children for selected parent. Source ordinals order is preserved. + * + * @lucene.experimental + */ +public final class TaxonomyChildrenOrdinalIterator implements OrdinalIterator { + + // TODO: do we want to have something like ChainOrdinalIterators to chain multiple iterators? + // Or are we fine with chaining them manually every time? + private final OrdinalIterator sourceOrds; + private final ParallelTaxonomyArrays.IntArray parents; + private final int parentOrd; + + /** Create */ + public TaxonomyChildrenOrdinalIterator( + OrdinalIterator sourceOrds, ParallelTaxonomyArrays.IntArray parents, int parentOrd) { + this.sourceOrds = sourceOrds; + this.parents = parents; + assert parentOrd != LabelToOrd.INVALID_ORD : "Parent Ordinal is not valid"; + this.parentOrd = parentOrd; + } + + @Override + public int nextOrd() throws IOException { + // TODO: in some cases it might be faster to traverse children of selected parent + // (children/siblings IntArrays) and check if source ords contain them. We can think of some + // heuristics to decide which approach to use on case by case basis? There is similar comment in + // TaxonomyFacets#getTopChildrenForPath + for (int ord = sourceOrds.nextOrd(); ord != NO_MORE_ORDS; ord = sourceOrds.nextOrd()) { + if (parents.get(ord) == parentOrd) { + return ord; + } + } + return NO_MORE_ORDS; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TopnOrdinalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TopnOrdinalIterator.java new file mode 100644 index 00000000000..684afec5f5d --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/TopnOrdinalIterator.java @@ -0,0 +1,113 @@ +/* + * 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.sandbox.facet.iterators; + +import java.io.IOException; +import org.apache.lucene.util.PriorityQueue; + +/** + * Class that consumes incoming ordinals, sorts them by provided Comparable, and returns first top N + * ordinals only. + * + * @lucene.experimental + */ +public final class TopnOrdinalIterator> implements OrdinalIterator { + + private final ComparableSupplier comparableSupplier; + private final OrdinalIterator sourceOrds; + private final int topN; + private int[] result; + private int currentIndex; + + /** Constructor. */ + public TopnOrdinalIterator( + OrdinalIterator sourceOrds, ComparableSupplier comparableSupplier, int topN) { + if (topN <= 0) { + throw new IllegalArgumentException("topN must be > 0 (got: " + topN + ")"); + } + this.sourceOrds = sourceOrds; + this.comparableSupplier = comparableSupplier; + this.topN = topN; + } + + private void getTopN() throws IOException { + assert result == null; + // TODO: current taxonomy implementations limit queue size by taxo reader size too, but it + // probably doesn't make sense for large enough taxonomy indexes? + // e.g. TopOrdAndIntQueue q = new TopComparableQueue(Math.min(taxoReader.getSize(), topN)); + // TODO: create queue lazily - skip if first nextOrd is NO_MORE_ORDS ? + TopComparableQueue queue = new TopComparableQueue<>(topN); + OrdComparablePair reuse = null; + for (int ord = sourceOrds.nextOrd(); ord != NO_MORE_ORDS; ord = sourceOrds.nextOrd()) { + if (reuse == null) { + reuse = new OrdComparablePair<>(ord, comparableSupplier.createComparable(ord)); + } else { + reuse.ordinal = ord; + comparableSupplier.reuseComparable(ord, reuse.comparable); + } + reuse = queue.insertWithOverflow(reuse); + } + // Now we need to read from the queue as well as the queue gives the least element, not the top. + result = new int[queue.size()]; + for (int i = result.length - 1; i >= 0; i--) { + result[i] = queue.pop().ordinal; + } + currentIndex = 0; + } + + @Override + public int nextOrd() throws IOException { + if (result == null) { + getTopN(); + } + assert result != null; + if (currentIndex >= result.length) { + return NO_MORE_ORDS; + } + return result[currentIndex++]; + } + + /** Keeps top N results ordered by Comparable. */ + private static class TopComparableQueue> + extends PriorityQueue> { + + /** Sole constructor. */ + public TopComparableQueue(int topN) { + super(topN); + } + + @Override + protected boolean lessThan(OrdComparablePair a, OrdComparablePair b) { + return a.lessThan(b); + } + } + + /** Pair of ordinal and comparable to use in TopComparableQueue */ + private static class OrdComparablePair> { + int ordinal; + T comparable; + + private OrdComparablePair(int ordinal, T comparable) { + this.ordinal = ordinal; + this.comparable = comparable; + } + + boolean lessThan(OrdComparablePair other) { + return comparable.compareTo(other.comparable) < 0; + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/package-info.java new file mode 100644 index 00000000000..c80222fedfc --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/iterators/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Sandbox faceting: facet ordinals. + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet.iterators; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/LabelToOrd.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/LabelToOrd.java new file mode 100644 index 00000000000..7bc537429d3 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/LabelToOrd.java @@ -0,0 +1,45 @@ +/* + * 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.sandbox.facet.labels; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.FacetLabel; + +/** + * Label to ord mapping interface. + * + *

    TODO: move FacetLabel out of taxonomy folder to use it for any facets, not just taxonomy? + * + *

    TODO: there is some overlap with {@link + * org.apache.lucene.facet.taxonomy.writercache.LabelToOrdinal}, can we reuse something? + * + * @lucene.experimental + */ +public interface LabelToOrd { + + /** + * Ordinal to return if facet label doesn't exist in {@link #getOrd(FacetLabel)} and {@link + * #getOrds(FacetLabel[])} + */ + int INVALID_ORD = -1; + + /** get ord for one label */ + int getOrd(FacetLabel label) throws IOException; + + /** get ords for multiple labels */ + int[] getOrds(FacetLabel[] labels) throws IOException; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/OrdToLabel.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/OrdToLabel.java new file mode 100644 index 00000000000..88e59844425 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/OrdToLabel.java @@ -0,0 +1,35 @@ +/* + * 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.sandbox.facet.labels; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.FacetLabel; + +/** + * Ordinal to label mapping interface. + * + *

    TODO: move FacetLabel out of taxonomy folder to use it for any facets, not just taxonomy? + * + * @lucene.experimental + */ +public interface OrdToLabel { + /** get label of one ord TODO: what do we return when ordinal is not valid? null? */ + FacetLabel getLabel(int ordinal) throws IOException; + + /** get labels for multiple ords */ + FacetLabel[] getLabels(int[] ordinals) throws IOException; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdToLabel.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdToLabel.java new file mode 100644 index 00000000000..3c0da42fbf4 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdToLabel.java @@ -0,0 +1,52 @@ +/* + * 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.sandbox.facet.labels; + +import org.apache.lucene.facet.range.Range; +import org.apache.lucene.facet.taxonomy.FacetLabel; + +/** + * {@link OrdToLabel} for ranges. + * + * @lucene.experimental + */ +public class RangeOrdToLabel implements OrdToLabel { + + private final Range[] ranges; + + /** Constructor that takes array of Range objects as input */ + public RangeOrdToLabel(Range[] inputRanges) { + ranges = inputRanges; + } + + @Override + public FacetLabel getLabel(int ordinal) { + if (ordinal >= 0 && ordinal < ranges.length) { + return new FacetLabel(ranges[ordinal].label); + } + return null; + } + + @Override + public FacetLabel[] getLabels(int[] ordinals) { + FacetLabel[] facetLabels = new FacetLabel[ordinals.length]; + for (int i = 0; i < ordinals.length; i++) { + facetLabels[i] = getLabel(ordinals[i]); + } + return facetLabels; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/TaxonomyOrdLabelBiMap.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/TaxonomyOrdLabelBiMap.java new file mode 100644 index 00000000000..e57df7cc139 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/TaxonomyOrdLabelBiMap.java @@ -0,0 +1,57 @@ +/* + * 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.sandbox.facet.labels; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; + +/** + * Map taxonomy labels to ordinals. + * + * @lucene.experimental + */ +public final class TaxonomyOrdLabelBiMap implements OrdToLabel, LabelToOrd { + + private final TaxonomyReader taxoReader; + + /** Construct */ + public TaxonomyOrdLabelBiMap(TaxonomyReader taxoReader) { + this.taxoReader = taxoReader; + } + + @Override + public FacetLabel getLabel(int ordinal) throws IOException { + return taxoReader.getPath(ordinal); + } + + @Override + public FacetLabel[] getLabels(int[] ordinals) throws IOException { + return taxoReader.getBulkPath( + ordinals.clone()); // Have to clone because getBulkPath shuffles its input array. + } + + @Override + public int getOrd(FacetLabel label) throws IOException { + return taxoReader.getOrdinal(label); + } + + @Override + public int[] getOrds(FacetLabel[] labels) throws IOException { + return taxoReader.getBulkOrdinals(labels); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/package-info.java new file mode 100644 index 00000000000..8d4c1c725f0 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ +/** + * Sandbox faceting: facet labels, see {@link org.apache.lucene.sandbox.facet.labels.OrdToLabel} for + * details. + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet.labels; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/package-info.java new file mode 100644 index 00000000000..f0e116bf68c --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/package-info.java @@ -0,0 +1,30 @@ +/* + * 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. + */ +/** + * Sandbox faceting - Collectors that compute facets. Facet Ordinals/Ids: Each doc may have + * different facets and therefore, different facet ordinals. For example a book can have Author, + * Publish Date, Page Count etc. as facets. The specific value for each of these Facets for a book + * can be mapped to an ordinal. Facet ordinals may be common across different book documents. + * FacetCutter: Can interpret Facets of a specific type for a doc type and output all the Facet + * Ordinals for the type for the doc. Facet Recorders: record data per ordinal. Some recorders may + * compute aggregations and record per ordinal data aggregated across an index. + * + *

    See SandboxFacetsExample for examples. + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java new file mode 100644 index 00000000000..0fab6dfb663 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java @@ -0,0 +1,169 @@ +/* + * 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.sandbox.facet.recorders; + +import static org.apache.lucene.sandbox.facet.iterators.OrdinalIterator.NO_MORE_ORDS; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.internal.hppc.IntCursor; +import org.apache.lucene.internal.hppc.IntIntHashMap; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * {@link FacetRecorder} to count facets. + * + *

    TODO: add an option to keep counts in an array, to improve performance for facets with small + * number of ordinals e.g. range facets. Options: - {@link LeafFacetCutter} can inform {@link + * LeafFacetRecorder} about expected number of facet ordinals ({@link + * org.apache.lucene.sandbox.facet.FacetFieldCollector} can orchestrate that). If expeted facet ord + * number is below some threshold - use array instead of a map? - first 100/1k counts in array, the + * rest - in a map; the limit can also be provided in a constructor? It is similar to what + * LongValuesFacetCounts does today. + * + *

    TODO: We can also consider collecting 2 (3, 4, ..., can be parametrizes) slices to a single + * sync map which can reduce thread contention compared to single sync map for all slices; at the + * same time there will be less work for reduce method. So far reduce wasn't a bottleneck for us, + * but it is definitely not free. + * + *

    TODO: If we come back to some for of synchronized count maps, we should be more careful what + * we acquire locks for - we used to lock addTo method itself, but it could be faster if we only + * synchronized after computing the key's hash; or we can lock the entire map only if we need to + * insert key, and lock single key otherwise? + * + * @lucene.experimental + */ +public final class CountFacetRecorder implements FacetRecorder { + private IntIntHashMap values; + private final List perLeafValues; + + /** Create. */ + public CountFacetRecorder() { + // Has to be synchronizedList as we have one recorder per all slices. + perLeafValues = Collections.synchronizedList(new ArrayList<>()); + } + + /** Get count for provided ordinal. */ + public int getCount(int ord) { + return values.get(ord); + } + + @Override + public LeafFacetRecorder getLeafRecorder(LeafReaderContext context) { + // TODO: we are planning to do some experiments with how hash maps are assigned to leaf or slice + // recorders, see other TODOs in this class. When we make the decision, we can collect + // leaf/slice recorders themselves, not the hashmaps? + IntIntHashMap leafValues = new IntIntHashMap(); + perLeafValues.add(leafValues); + return new CountLeafFacetRecorder(leafValues); + } + + @Override + public OrdinalIterator recordedOrds() { + // TODO: even if this is called before collection started, we want it to use results from the + // time when nextOrd is first called. Does ordIterator work like that? I've run some tests that + // confirmed expected behavior, but I'm not sure IntIntMap guarantees that. We should at least + // add a unit test to make sure it always work that way. + Iterator ordIterator = values.keys().iterator(); + return new OrdinalIterator() { + @Override + public int nextOrd() { + if (ordIterator.hasNext()) { + return ordIterator.next().value; + } else { + return NO_MORE_ORDS; + } + } + }; + } + + @Override + public boolean isEmpty() { + return values.isEmpty(); + } + + @Override + public void reduce(FacetCutter facetCutter) throws IOException { + boolean firstElement = true; + for (IntIntHashMap leafRecords : perLeafValues) { + if (firstElement) { + values = leafRecords; + firstElement = false; + } else { + for (IntIntHashMap.IntIntCursor elem : leafRecords) { + values.addTo(elem.key, elem.value); + } + } + } + if (firstElement) { + // TODO: do we need empty map by default? + values = new IntIntHashMap(); + } + + OrdinalIterator dimOrds = facetCutter.getOrdinalsToRollup(); + if (dimOrds != null) { + for (int dimOrd = dimOrds.nextOrd(); dimOrd != NO_MORE_ORDS; dimOrd = dimOrds.nextOrd()) { + int rolledUp = rollup(dimOrd, facetCutter); + if (rolledUp > 0) { + values.addTo(dimOrd, rolledUp); + } + } + } + } + + @Override + public boolean contains(int ordinal) { + return values.containsKey(ordinal); + } + + private int rollup(int ord, FacetCutter facetCutter) throws IOException { + OrdinalIterator childOrds = facetCutter.getChildrenOrds(ord); + int accum = 0; + for (int nextChild = childOrds.nextOrd(); + nextChild != NO_MORE_ORDS; + nextChild = childOrds.nextOrd()) { + int rolledUp = rollup(nextChild, facetCutter); + // Don't rollup zeros to not add ordinals that we don't actually have counts for to the map + if (rolledUp > 0) { + accum += values.addTo(nextChild, rolledUp); + } else { + accum += values.get(nextChild); + } + } + return accum; + } + + private static class CountLeafFacetRecorder implements LeafFacetRecorder { + + private final IntIntHashMap values; + + public CountLeafFacetRecorder(IntIntHashMap values) { + this.values = values; + } + + @Override + public void record(int docId, int facetOrd) { + this.values.addTo(facetOrd, 1); + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/FacetRecorder.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/FacetRecorder.java new file mode 100644 index 00000000000..f86fe7bf8de --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/FacetRecorder.java @@ -0,0 +1,63 @@ +/* + * 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.sandbox.facet.recorders; + +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * Record data for each facet of each doc. + * + *

    TODO: In the next iteration we can add an extra layer between FacetRecorder and + * LeafFacetRecorder, e.g. SliceFacetRecorder. The new layer will be created per {@link + * org.apache.lucene.search.Collector}, which means that collecting of multiple leafs (segments) + * within a slice is sequential and can be done to a single non-sync map to improve performance and + * reduce memory consumption. We already tried that, but didn't see any performance improvement. + * Given that it also makes lazy leaf recorder init in {@link + * org.apache.lucene.sandbox.facet.FacetFieldCollector} trickier, it was decided to rollback the + * initial attempt and try again later, in the next iteration. + * + * @lucene.experimental + */ +public interface FacetRecorder { + /** Get leaf recorder. */ + LeafFacetRecorder getLeafRecorder(LeafReaderContext context) throws IOException; + + /** Return next collected ordinal, or {@link LeafFacetCutter#NO_MORE_ORDS} */ + OrdinalIterator recordedOrds(); + + /** True if there are no records */ + boolean isEmpty(); + + /** + * Reduce leaf recorder results into this recorder. If {@link FacetCutter#getOrdinalsToRollup()} + * result is not null, it also rolls up values. + * + *

    After this method is called, it's illegal to add values to recorder, i.e. calling {@link + * #getLeafRecorder} or {@link LeafFacetRecorder#record} on its leaf recorders. + * + * @throws UnsupportedOperationException if {@link FacetCutter#getOrdinalsToRollup()} returns not + * null but this recorder doesn't support rollup. + */ + void reduce(FacetCutter facetCutter) throws IOException; + + /** Check if any data was recorded for provided facet ordinal. */ + boolean contains(int ordinal); +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LeafFacetRecorder.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LeafFacetRecorder.java new file mode 100644 index 00000000000..eff0f01c8ff --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LeafFacetRecorder.java @@ -0,0 +1,35 @@ +/* + * 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.sandbox.facet.recorders; + +import java.io.IOException; + +/** + * Record data for each facet of each doc of a leaf (segment). + * + * @lucene.experimental + */ +public interface LeafFacetRecorder { + + /** + * TODO: Rename: collect? accumulate? + * + * @param docId document ID + * @param facetOrd facet ordinal + */ + void record(int docId, int facetOrd) throws IOException; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java new file mode 100644 index 00000000000..c7f475b4fdb --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java @@ -0,0 +1,207 @@ +/* + * 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.sandbox.facet.recorders; + +import static org.apache.lucene.sandbox.facet.iterators.OrdinalIterator.NO_MORE_ORDS; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.internal.hppc.IntCursor; +import org.apache.lucene.internal.hppc.IntObjectHashMap; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.search.LongValues; +import org.apache.lucene.search.LongValuesSource; + +/** + * {@link FacetRecorder} that computes multiple long aggregations per facet. + * + *

    TODO: [premature optimization idea] if instead of one array we keep aggregations in two + * LongVector (one for MAX aggregation and one for SUM) we can benefit from SIMD? + * + * @lucene.experimental + */ +public final class LongAggregationsFacetRecorder implements FacetRecorder { + + private IntObjectHashMap values; + private final List> leafValues; + + private final LongValuesSource[] longValuesSources; + private final Reducer[] reducers; + + /** Constructor. */ + public LongAggregationsFacetRecorder(LongValuesSource[] longValuesSources, Reducer[] reducers) { + assert longValuesSources.length == reducers.length; + this.longValuesSources = longValuesSources; + this.reducers = reducers; + leafValues = Collections.synchronizedList(new ArrayList<>()); + } + + @Override + public LeafFacetRecorder getLeafRecorder(LeafReaderContext context) throws IOException { + LongValues[] longValues = new LongValues[longValuesSources.length]; + for (int i = 0; i < longValuesSources.length; i++) { + longValues[i] = longValuesSources[i].getValues(context, null); + } + IntObjectHashMap valuesRecorder = new IntObjectHashMap<>(); + leafValues.add(valuesRecorder); + return new LongAggregationsLeafFacetRecorder(longValues, reducers, valuesRecorder); + } + + @Override + public OrdinalIterator recordedOrds() { + Iterator ordIterator = values.keys().iterator(); + return new OrdinalIterator() { + @Override + public int nextOrd() throws IOException { + if (ordIterator.hasNext()) { + return ordIterator.next().value; + } else { + return NO_MORE_ORDS; + } + } + }; + } + + @Override + public boolean isEmpty() { + return values.isEmpty(); + } + + @Override + public void reduce(FacetCutter facetCutter) throws IOException { + boolean firstElement = true; + for (IntObjectHashMap leafValue : leafValues) { + if (firstElement) { + values = leafValue; + firstElement = false; + } else { + for (IntObjectHashMap.IntObjectCursor elem : leafValue) { + long[] vals = values.get(elem.key); + if (vals == null) { + values.put(elem.key, elem.value); + } else { + for (int i = 0; i < longValuesSources.length; i++) { + vals[i] = reducers[i].reduce(vals[i], elem.value[i]); + } + } + } + } + } + if (firstElement) { + // TODO: do we need empty map by default? + values = new IntObjectHashMap<>(); + } + + OrdinalIterator dimOrds = facetCutter.getOrdinalsToRollup(); + if (dimOrds != null) { + for (int dimOrd = dimOrds.nextOrd(); dimOrd != NO_MORE_ORDS; dimOrd = dimOrds.nextOrd()) { + rollup(values.get(dimOrd), dimOrd, facetCutter); + } + } + } + + @Override + public boolean contains(int ordinal) { + return values.containsKey(ordinal); + } + + /** + * Rollup all child values of ord to accum, and return accum. Accum param can be null. In this + * case, if recursive rollup for every child returns null, this method returns null. Otherwise, + * accum is initialized. + */ + private long[] rollup(long[] accum, int ord, FacetCutter facetCutter) throws IOException { + OrdinalIterator childOrds = facetCutter.getChildrenOrds(ord); + for (int nextChild = childOrds.nextOrd(); + nextChild != NO_MORE_ORDS; + nextChild = childOrds.nextOrd()) { + long[] current = rollup(values.get(nextChild), nextChild, facetCutter); + if (current != null) { + if (accum == null) { + accum = new long[longValuesSources.length]; + values.put(ord, accum); + } + for (int i = 0; i < longValuesSources.length; i++) { + accum[i] = reducers[i].reduce(accum[i], current[i]); + } + } + } + return accum; + } + + /** Return aggregated value for facet ordinal and aggregation ID, or zero as default. */ + public long getRecordedValue(int ord, int valuesId) { + if (valuesId < 0 || valuesId >= longValuesSources.length) { + throw new IllegalArgumentException("Invalid request for ordinal values"); + } + long[] valuesForOrd = values.get(ord); + if (valuesForOrd != null) { + return valuesForOrd[valuesId]; + } + // There are a few options what we can return here e.g. throw an exception, return hardcoded or + // provided default value. It might be better API to do that instead of returning zero, but + // there are two reasons why I think returning zero is the right compromise: + // 1) recorder result is a map-like structure, and maps in java usually return default value + // e.g. null or 0 rather than throw an exception when a key is missing. + // 2) Handling correctly all missing value cases might be expensive, e.g. what if only one + // aggregation for selected facet ordinal is missing, i.e. no docs that belong to this facet + // ordinal have a value to aggregate? To handle that we would have to maintain missing values + // during collection instead of using default array value - zero. I believe it is excessive and + // most users are not going to use it anyway. Worst case scenario, we can add another public get + // method that handles missing values later. + return 0; + } + + private static class LongAggregationsLeafFacetRecorder implements LeafFacetRecorder { + + private final LongValues[] longValues; + + private final Reducer[] reducers; + private final IntObjectHashMap perOrdinalValues; + + LongAggregationsLeafFacetRecorder( + LongValues[] longValues, Reducer[] reducers, IntObjectHashMap perOrdinalValues) { + this.longValues = longValues; + this.reducers = reducers; + this.perOrdinalValues = perOrdinalValues; + } + + @Override + public void record(int docId, int facetOrd) throws IOException { + long[] valuesForOrd = perOrdinalValues.get(facetOrd); + if (valuesForOrd == null) { + valuesForOrd = new long[longValues.length]; + perOrdinalValues.put(facetOrd, valuesForOrd); + } + + LongValues values; + for (int i = 0; i < longValues.length; i++) { + // TODO: cache advance/longValue results for current doc? Skipped for now as LongValues + // themselves can keep the cache. + values = longValues[i]; + if (values.advanceExact(docId)) { + valuesForOrd[i] = reducers[i].reduce(valuesForOrd[i], values.longValue()); + } + } + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/MultiFacetsRecorder.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/MultiFacetsRecorder.java new file mode 100644 index 00000000000..db336e38207 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/MultiFacetsRecorder.java @@ -0,0 +1,87 @@ +/* + * 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.sandbox.facet.recorders; + +import java.io.IOException; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; + +/** + * {@link FacetRecorder} that contains multiple FacetRecorders. + * + * @lucene.experimental + */ +public final class MultiFacetsRecorder implements FacetRecorder { + + private final FacetRecorder[] delegates; + + /** Constructor */ + public MultiFacetsRecorder(FacetRecorder... delegates) { + this.delegates = delegates; + } + + @Override + public LeafFacetRecorder getLeafRecorder(LeafReaderContext context) throws IOException { + LeafFacetRecorder[] leafDelegates = new LeafFacetRecorder[delegates.length]; + for (int i = 0; i < delegates.length; i++) { + leafDelegates[i] = delegates[i].getLeafRecorder(context); + } + return new MultiFacetsLeafRecorder(leafDelegates); + } + + @Override + public OrdinalIterator recordedOrds() { + throw new UnsupportedOperationException( + "Not supported, call recordedOrds for sub-recorders instead"); + } + + @Override + public boolean isEmpty() { + throw new UnsupportedOperationException( + "Not supported, call isEmpty for sub-recorders instead"); + } + + @Override + public void reduce(FacetCutter facetCutter) throws IOException { + for (FacetRecorder recorder : delegates) { + recorder.reduce(facetCutter); + } + } + + @Override + public boolean contains(int ordinal) { + throw new UnsupportedOperationException( + "Not supported, call contains for sub-recorders instead"); + } + + private static final class MultiFacetsLeafRecorder implements LeafFacetRecorder { + + private final LeafFacetRecorder[] delegates; + + private MultiFacetsLeafRecorder(LeafFacetRecorder[] delegates) { + this.delegates = delegates; + } + + @Override + public void record(int docId, int facetOrd) throws IOException { + for (LeafFacetRecorder leafRecorder : delegates) { + leafRecorder.record(docId, facetOrd); + } + } + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/Reducer.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/Reducer.java new file mode 100644 index 00000000000..b10e20dce18 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/Reducer.java @@ -0,0 +1,85 @@ +/* + * 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.sandbox.facet.recorders; + +/** + * Reducer for numeric values. + * + * @lucene.experimental + */ +public interface Reducer { + + /** Int values reducer. */ + int reduce(int a, int b); + + /** Long values reducer. */ + long reduce(long a, long b); + + /** Float values reducer. */ + float reduce(float a, float b); + + /** Double values reducer. */ + double reduce(double a, double b); + + /** Reducer that returns MAX of two values. */ + Reducer MAX = + new Reducer() { + @Override + public int reduce(int a, int b) { + return Math.max(a, b); + } + + @Override + public long reduce(long a, long b) { + return Math.max(a, b); + } + + @Override + public float reduce(float a, float b) { + return Math.max(a, b); + } + + @Override + public double reduce(double a, double b) { + return Math.max(a, b); + } + }; + + /** Reducer that returns SUM of two values. */ + Reducer SUM = + new Reducer() { + @Override + public int reduce(int a, int b) { + return Math.addExact(a, b); + } + + @Override + public long reduce(long a, long b) { + return Math.addExact(a, b); + } + + @Override + public float reduce(float a, float b) { + return a + b; + } + + @Override + public double reduce(double a, double b) { + return a + b; + } + }; +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/package-info.java new file mode 100644 index 00000000000..26c35d50575 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ +/** + * Sandbox faceting: classes that can record per ordinal data E.g. aggregations per facet ordinals + * can be recorded. + * + * @lucene.experimental + */ +package org.apache.lucene.sandbox.facet.recorders; diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/SandboxFacetTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/SandboxFacetTestCase.java new file mode 100644 index 00000000000..560a3d4bc30 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/SandboxFacetTestCase.java @@ -0,0 +1,198 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.sandbox.facet.iterators.CandidateSetOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.ComparableSupplier; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TaxonomyChildrenOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TopnOrdinalIterator; +import org.apache.lucene.sandbox.facet.labels.OrdToLabel; +import org.apache.lucene.sandbox.facet.labels.TaxonomyOrdLabelBiMap; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.tests.util.LuceneTestCase; + +public abstract class SandboxFacetTestCase extends LuceneTestCase { + // TODO: We don't have access to overall count for all facets from count recorder, and we can't + // compute it as a SUM of values for each facet ordinal because we need to respect cases where the + // same doc belongs to multiple facets (e.g. overlapping ranges or multi value fields). In most + // cases we can already access the value. E.g. for facets with hierarchy (taxonomy or SSDV) we can + // read value for parent facet ordinal. I believe the only case that requires code changes is + // range facets. To solve it we can add a parameter to range FacetCutter to assign/yeild special + // facet ordinal for every document that matches at least one range from the list. Overall, + // sandbox facet tests don't have to use FacetResult, so we change it to assert facet labels and + // recorded results directly and avoid need for this constant. + static final int VALUE_CANT_BE_COMPUTED = Integer.MIN_VALUE; + + protected void assertNumericValuesEquals(Number a, Number b) { + assertTrue(a.getClass().isInstance(b)); + if (a instanceof Float) { + assertEquals(a.floatValue(), b.floatValue(), a.floatValue() / 1e5); + } else if (a instanceof Double) { + assertEquals(a.doubleValue(), b.doubleValue(), a.doubleValue() / 1e5); + } else { + assertEquals(a, b); + } + } + + protected void assertFacetResult( + FacetResult result, + String expectedDim, + String[] expectedPath, + int expectedChildCount, + Number expectedValue, + LabelAndValue... expectedChildren) { + assertEquals(expectedDim, result.dim); + assertArrayEquals(expectedPath, result.path); + assertEquals(expectedChildCount, result.childCount); + assertNumericValuesEquals(expectedValue, result.value); + assertEquals(expectedChildren.length, result.labelValues.length); + // assert children equal with no assumption of the children ordering + assertTrue(Arrays.asList(result.labelValues).containsAll(Arrays.asList(expectedChildren))); + } + + FacetResult getTopChildrenByCount( + CountFacetRecorder countFacetRecorder, + TaxonomyReader taxoReader, + int topN, + String dimension, + String... path) + throws IOException { + ComparableSupplier countComparable = + ComparableUtils.byCount(countFacetRecorder); + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + FacetLabel parentLabel = new FacetLabel(dimension, path); + OrdinalIterator childrenIterator = + new TaxonomyChildrenOrdinalIterator( + countFacetRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + ordLabels.getOrd(parentLabel)); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIterator, countComparable, topN); + // Get array of final ordinals - we need to use all of them to get labels first, and then to get + // counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = topByCountOrds.toArray(); + + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countFacetRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return new FacetResult( + dimension, + path, + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } + + FacetResult getAllChildren( + CountFacetRecorder countFacetRecorder, + TaxonomyReader taxoReader, + String dimension, + String... path) + throws IOException { + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + FacetLabel parentLabel = new FacetLabel(dimension, path); + int parentOrdinal = ordLabels.getOrd(parentLabel); + OrdinalIterator childrenIternator = + new TaxonomyChildrenOrdinalIterator( + countFacetRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + parentOrdinal); + // Get array of final ordinals - we need to use all of them to get labels first, and then to get + // counts, + // but OrdinalIterator only allows reading ordinals once. + int[] resultOrdinals = childrenIternator.toArray(); + + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countFacetRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return new FacetResult( + dimension, + path, + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } + + FacetResult getAllSortByOrd( + int[] resultOrdinals, + CountFacetRecorder countFacetRecorder, + String dimension, + OrdToLabel ordLabels) + throws IOException { + ComparableUtils.sort(resultOrdinals, ComparableUtils.byOrdinal()); + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countFacetRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + + return new FacetResult( + dimension, + new String[0], + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } + + int getSpecificValue( + CountFacetRecorder countFacetRecorder, TaxonomyReader taxoReader, String... path) + throws IOException { + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + FacetLabel label = new FacetLabel(path); + int facetOrd = ordLabels.getOrd(label); + return countFacetRecorder.getCount(facetOrd); + } + + int[] getCountsForRecordedCandidates( + CountFacetRecorder countFacetRecorder, TaxonomyReader taxoReader, FacetLabel[] candidates) + throws IOException { + int[] resultOrds = + new CandidateSetOrdinalIterator( + countFacetRecorder, candidates, new TaxonomyOrdLabelBiMap(taxoReader)) + .toArray(); + int[] counts = new int[resultOrds.length]; + for (int i = 0; i < resultOrds.length; i++) { + counts[i] = countFacetRecorder.getCount(resultOrds[i]); + } + return counts; + } +} diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestCandidateSetOrdinalIterator.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestCandidateSetOrdinalIterator.java new file mode 100644 index 00000000000..5b8f55b96a5 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestCandidateSetOrdinalIterator.java @@ -0,0 +1,100 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.FacetCutter; +import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.CandidateSetOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.sandbox.facet.labels.LabelToOrd; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.LeafFacetRecorder; +import org.apache.lucene.tests.util.LuceneTestCase; + +/** Tests for {@link CandidateSetOrdinalIterator}. */ +public class TestCandidateSetOrdinalIterator extends LuceneTestCase { + + /** LabelToOrd that parses label's string to get int ordinal */ + private LabelToOrd mockLabelToOrd = + new LabelToOrd() { + @Override + public int getOrd(FacetLabel label) { + return Integer.valueOf(label.lastComponent()); + } + + @Override + public int[] getOrds(FacetLabel[] labels) { + int[] result = new int[labels.length]; + for (int i = 0; i < result.length; i++) { + result[i] = getOrd(labels[i]); + } + return result; + } + }; + + private FacetCutter mockFacetCutter = + new FacetCutter() { + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + return null; + } + }; + + public void testBasic() throws IOException { + FacetRecorder recorder = new CountFacetRecorder(); + LeafFacetRecorder leafRecorder = recorder.getLeafRecorder(null); + leafRecorder.record(0, 0); + leafRecorder.record(0, 3); + recorder.reduce(mockFacetCutter); + + FacetLabel[] candidates = + new FacetLabel[] { + new FacetLabel("0"), + new FacetLabel("1"), + new FacetLabel(String.valueOf(LabelToOrd.INVALID_ORD)), + new FacetLabel("3") + }; + + // Note that "1" is filtered out as it was never recorded + assertArrayEquals( + new int[] {0, 3}, + new CandidateSetOrdinalIterator(recorder, candidates, mockLabelToOrd).toArray()); + } + + public void testEmptyRecorder() throws IOException { + FacetRecorder recorder = new CountFacetRecorder(); + recorder.reduce(mockFacetCutter); + + FacetLabel[] candidates = + new FacetLabel[] { + new FacetLabel("0"), + new FacetLabel("1"), + new FacetLabel(String.valueOf(LabelToOrd.INVALID_ORD)), + new FacetLabel("3") + }; + + // Note that "1" is filtered out as it was never recorded + assertEquals( + OrdinalIterator.NO_MORE_ORDS, + new CandidateSetOrdinalIterator(recorder, candidates, mockLabelToOrd).nextOrd()); + } +} diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestFacetRecorders.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestFacetRecorders.java new file mode 100644 index 00000000000..41d73e16f98 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestFacetRecorders.java @@ -0,0 +1,478 @@ +/* + * 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.sandbox.facet; + +import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoubleDocValuesField; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.facet.FacetField; +import org.apache.lucene.facet.FacetsConfig; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.Term; +import org.apache.lucene.sandbox.facet.cutters.TaxonomyFacetsCutter; +import org.apache.lucene.sandbox.facet.iterators.CandidateSetOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.ComparableSupplier; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TaxonomyChildrenOrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TopnOrdinalIterator; +import org.apache.lucene.sandbox.facet.labels.TaxonomyOrdLabelBiMap; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.FacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder; +import org.apache.lucene.sandbox.facet.recorders.MultiFacetsRecorder; +import org.apache.lucene.sandbox.facet.recorders.Reducer; +import org.apache.lucene.search.DoubleValuesSource; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LongValuesSource; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.IOUtils; + +/** Test for {@link FacetRecorder} */ +public class TestFacetRecorders extends SandboxFacetTestCase { + + public void testCountAndLongAggregationRecordersBasic() throws Exception { + Directory dir = newDirectory(); + Directory taxoDir = newDirectory(); + + // Writes facet ords to a separate directory from the + // main index: + DirectoryTaxonomyWriter taxoWriter = + new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + config.setHierarchical("Publish Date", true); + config.setMultiValued("Publish Date", random().nextBoolean()); + + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new FacetField("Author", "Bob")); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + doc.add(new NumericDocValuesField("Units", 9)); + doc.add(new DoubleDocValuesField("Popularity", 3.5d)); + doc.add(new StringField("Availability", "yes", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2010")); + doc.add(new NumericDocValuesField("Units", 2)); + doc.add(new DoubleDocValuesField("Popularity", 4.1D)); + doc.add(new StringField("Availability", "yes", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2012", "1", "1")); + doc.add(new NumericDocValuesField("Units", 5)); + doc.add(new DoubleDocValuesField("Popularity", 3.9D)); + doc.add(new StringField("Availability", "yes", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Susan")); + doc.add(new FacetField("Publish Date", "2012", "1", "7")); + doc.add(new NumericDocValuesField("Units", 7)); + doc.add(new DoubleDocValuesField("Popularity", 4D)); + doc.add(new StringField("Availability", "yes", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Frank")); + doc.add(new FacetField("Publish Date", "1999", "5", "5")); + doc.add(new NumericDocValuesField("Units", 6)); + doc.add(new DoubleDocValuesField("Popularity", 7.9D)); + doc.add(new StringField("Availability", "yes", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + // Add a document that is not returned by a query + doc = new Document(); + doc.add(new FacetField("Author", "John")); + doc.add(new FacetField("Publish Date", "2024", "11", "12")); + doc.add(new NumericDocValuesField("Units", 200)); + doc.add(new DoubleDocValuesField("Popularity", 13D)); + doc.add(new StringField("Availability", "no", Field.Store.NO)); + writer.addDocument(config.build(taxoWriter, doc)); + + // NRT open + IndexSearcher searcher = newSearcher(writer.getReader()); + + // NRT open + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter); + + Query query = new TermQuery(new Term("Availability", "yes")); + + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + + LongValuesSource[] longValuesSources = new LongValuesSource[2]; + Reducer[] reducers = new Reducer[2]; + // popularity:max + longValuesSources[0] = DoubleValuesSource.fromDoubleField("Popularity").toLongValuesSource(); + reducers[0] = Reducer.MAX; + // units:sum + longValuesSources[1] = LongValuesSource.fromLongField("Units"); + reducers[1] = Reducer.SUM; + + LongAggregationsFacetRecorder longAggregationsFacetRecorder = + new LongAggregationsFacetRecorder(longValuesSources, reducers); + + final CountFacetRecorder countRecorder = new CountFacetRecorder(); + // Compute both counts and aggregations + MultiFacetsRecorder multiFacetsRecorder = + new MultiFacetsRecorder(countRecorder, longAggregationsFacetRecorder); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, multiFacetsRecorder); + searcher.search(query, collectorManager); + + int[] ordsFromCounts = countRecorder.recordedOrds().toArray(); + Arrays.sort(ordsFromCounts); + int[] ordsFromAggregations = longAggregationsFacetRecorder.recordedOrds().toArray(); + Arrays.sort(ordsFromAggregations); + assertArrayEquals(ordsFromCounts, ordsFromAggregations); + + // Retrieve & verify results: + assertEquals( + "dim=Publish Date path=[]\n" + + " 2010 (2, agg0=4 agg1=11)\n" + + " 2012 (2, agg0=4 agg1=12)\n" + + " 1999 (1, agg0=7 agg1=6)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, null, "Publish Date")); + assertEquals( + "dim=Author path=[]\n" + + " Lisa (2, agg0=4 agg1=7)\n" + + " Bob (1, agg0=3 agg1=9)\n" + + " Susan (1, agg0=4 agg1=7)\n" + + " Frank (1, agg0=7 agg1=6)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, null, "Author")); + + assertArrayEquals( + new long[] {11, 6}, + getAggregationForRecordedCandidates( + longAggregationsFacetRecorder, + 1, + taxoReader, + new FacetLabel[] { + new FacetLabel("Publish Date", "2010"), + // Not in the index - skipped + new FacetLabel("Publish Date", "2025"), + // Not matched by the query - skipped + new FacetLabel("Publish Date", "2024"), + new FacetLabel("Publish Date", "1999"), + })); + + assertArrayEquals( + new long[] {7, 6}, + getAggregationForRecordedCandidates( + longAggregationsFacetRecorder, + 1, + taxoReader, + new FacetLabel[] { + new FacetLabel("Author", "Lisa"), + // Not in the index - skipped + new FacetLabel("Author", "Christofer"), + // Not matched by the query - skipped + new FacetLabel("Author", "John"), + new FacetLabel("Author", "Frank"), + })); + + writer.close(); + IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, taxoDir, dir); + } + + /** + * Test that counts and long aggregations are correct when different index segments have different + * facet ordinals. + */ + public void testCountAndLongAggregationRecordersMultipleSegments() throws Exception { + Directory dir = newDirectory(); + Directory taxoDir = newDirectory(); + + // Writes facet ords to a separate directory from the + // main index: + DirectoryTaxonomyWriter taxoWriter = + new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + config.setHierarchical("Publish Date", true); + config.setMultiValued("Publish Date", random().nextBoolean()); + + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new FacetField("Author", "Bob")); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + doc.add(new NumericDocValuesField("Units", 9)); + doc.add(new DoubleDocValuesField("Popularity", 3.5d)); + writer.addDocument(config.build(taxoWriter, doc)); + writer.commit(); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2012", "10", "20")); + doc.add(new NumericDocValuesField("Units", 2)); + doc.add(new DoubleDocValuesField("Popularity", 4.1D)); + writer.addDocument(config.build(taxoWriter, doc)); + writer.commit(); + + // NRT open + IndexSearcher searcher = newSearcher(writer.getReader()); + + // NRT open + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter); + + Query query = new MatchAllDocsQuery(); + + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + + LongValuesSource[] longValuesSources = new LongValuesSource[2]; + Reducer[] reducers = new Reducer[2]; + // popularity:max + longValuesSources[0] = DoubleValuesSource.fromDoubleField("Popularity").toLongValuesSource(); + reducers[0] = Reducer.MAX; + // units:sum + longValuesSources[1] = LongValuesSource.fromLongField("Units"); + reducers[1] = Reducer.SUM; + + LongAggregationsFacetRecorder longAggregationsFacetRecorder = + new LongAggregationsFacetRecorder(longValuesSources, reducers); + + final CountFacetRecorder countRecorder = new CountFacetRecorder(); + // Compute both counts and aggregations + MultiFacetsRecorder multiFacetsRecorder = + new MultiFacetsRecorder(countRecorder, longAggregationsFacetRecorder); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, multiFacetsRecorder); + searcher.search(query, collectorManager); + + // Retrieve & verify results: + assertEquals( + "dim=Publish Date path=[]\n" + + " 2010 (1, agg0=3 agg1=9)\n" + + " 2012 (1, agg0=4 agg1=2)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, null, "Publish Date")); + assertEquals( + "dim=Author path=[]\n" + " Bob (1, agg0=3 agg1=9)\n" + " Lisa (1, agg0=4 agg1=2)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, null, "Author")); + + writer.close(); + IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, taxoDir, dir); + } + + public void testSortByLongAggregation() throws Exception { + Directory dir = newDirectory(); + Directory taxoDir = newDirectory(); + + // Writes facet ords to a separate directory from the + // main index: + DirectoryTaxonomyWriter taxoWriter = + new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + config.setHierarchical("Publish Date", true); + config.setMultiValued("Publish Date", random().nextBoolean()); + + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new FacetField("Author", "Bob")); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + doc.add(new NumericDocValuesField("Units", 9)); + doc.add(new DoubleDocValuesField("Popularity", 3.5d)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2010", "10", "20")); + doc.add(new NumericDocValuesField("Units", 2)); + doc.add(new DoubleDocValuesField("Popularity", 4.1D)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2012", "1", "1")); + doc.add(new NumericDocValuesField("Units", 5)); + doc.add(new DoubleDocValuesField("Popularity", 3.9D)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Susan")); + doc.add(new FacetField("Publish Date", "2012", "1", "7")); + doc.add(new NumericDocValuesField("Units", 7)); + doc.add(new DoubleDocValuesField("Popularity", 4D)); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Frank")); + doc.add(new FacetField("Publish Date", "1999", "5", "5")); + doc.add(new NumericDocValuesField("Units", 6)); + doc.add(new DoubleDocValuesField("Popularity", 7.9D)); + writer.addDocument(config.build(taxoWriter, doc)); + + // NRT open + IndexSearcher searcher = newSearcher(writer.getReader()); + + // NRT open + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter); + + Query query = new MatchAllDocsQuery(); + + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + + LongValuesSource[] longValuesSources = new LongValuesSource[2]; + Reducer[] reducers = new Reducer[2]; + // popularity:max + longValuesSources[0] = DoubleValuesSource.fromDoubleField("Popularity").toLongValuesSource(); + reducers[0] = Reducer.MAX; + // units:sum + longValuesSources[1] = LongValuesSource.fromLongField("Units"); + reducers[1] = Reducer.SUM; + + LongAggregationsFacetRecorder longAggregationsFacetRecorder = + new LongAggregationsFacetRecorder(longValuesSources, reducers); + + final CountFacetRecorder countRecorder = new CountFacetRecorder(); + // Compute both counts and aggregations + MultiFacetsRecorder multiFacetsRecorder = + new MultiFacetsRecorder(countRecorder, longAggregationsFacetRecorder); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, multiFacetsRecorder); + searcher.search(query, collectorManager); + + // Retrieve & verify results: + assertEquals( + "dim=Publish Date path=[]\n" + + " 2012 (2, agg0=4 agg1=12)\n" + + " 2010 (2, agg0=4 agg1=11)\n" + + " 1999 (1, agg0=7 agg1=6)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, 1, "Publish Date")); + assertEquals( + "dim=Author path=[]\n" + + " Frank (1, agg0=7 agg1=6)\n" + + " Lisa (2, agg0=4 agg1=7)\n" + + " Susan (1, agg0=4 agg1=7)\n" + + " Bob (1, agg0=3 agg1=9)\n", + getTopChildrenWithLongAggregations( + countRecorder, taxoReader, 10, 2, longAggregationsFacetRecorder, 0, "Author")); + + writer.close(); + IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, taxoDir, dir); + } + + private String getTopChildrenWithLongAggregations( + CountFacetRecorder countFacetRecorder, + TaxonomyReader taxoReader, + int topN, + int numOfAggregations, + LongAggregationsFacetRecorder longAggregationsFacetRecorder, + Integer sortByLongAggregationId, + String dimension, + String... path) + throws IOException { + StringBuilder resultBuilder = new StringBuilder(); + resultBuilder.append("dim="); + resultBuilder.append(dimension); + resultBuilder.append(" path="); + resultBuilder.append(Arrays.toString(path)); + resultBuilder.append('\n'); + + TaxonomyOrdLabelBiMap ordLabels = new TaxonomyOrdLabelBiMap(taxoReader); + FacetLabel parentLabel = new FacetLabel(dimension, path); + OrdinalIterator childrenIternator = + new TaxonomyChildrenOrdinalIterator( + countFacetRecorder.recordedOrds(), + taxoReader.getParallelTaxonomyArrays().parents(), + ordLabels.getOrd(parentLabel)); + final int[] resultOrdinals; + if (sortByLongAggregationId != null) { + ComparableSupplier comparableSupplier = + ComparableUtils.byAggregatedValue( + countFacetRecorder, longAggregationsFacetRecorder, sortByLongAggregationId); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIternator, comparableSupplier, topN); + resultOrdinals = topByCountOrds.toArray(); + } else { + ComparableSupplier countComparable = + ComparableUtils.byCount(countFacetRecorder); + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(childrenIternator, countComparable, topN); + resultOrdinals = topByCountOrds.toArray(); + } + + FacetLabel[] labels = ordLabels.getLabels(resultOrdinals); + for (int i = 0; i < resultOrdinals.length; i++) { + int facetOrdinal = resultOrdinals[i]; + int count = countFacetRecorder.getCount(facetOrdinal); + resultBuilder.append(" "); + resultBuilder.append(labels[i].lastComponent()); + resultBuilder.append(" ("); + resultBuilder.append(count); + resultBuilder.append(", "); + for (int a = 0; a < numOfAggregations; a++) { + resultBuilder.append(" agg"); + resultBuilder.append(a); + resultBuilder.append("="); + resultBuilder.append(longAggregationsFacetRecorder.getRecordedValue(facetOrdinal, a)); + } + resultBuilder.append(")"); + resultBuilder.append("\n"); + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return resultBuilder.toString(); + } + + long[] getAggregationForRecordedCandidates( + LongAggregationsFacetRecorder aggregationsRecorder, + int aggregationId, + TaxonomyReader taxoReader, + FacetLabel[] candidates) + throws IOException { + int[] resultOrds = + new CandidateSetOrdinalIterator( + aggregationsRecorder, candidates, new TaxonomyOrdLabelBiMap(taxoReader)) + .toArray(); + long[] result = new long[resultOrds.length]; + for (int i = 0; i < resultOrds.length; i++) { + result[i] = aggregationsRecorder.getRecordedValue(resultOrds[i], aggregationId); + } + return result; + } +} diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestLongValueFacet.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestLongValueFacet.java new file mode 100644 index 00000000000..2c2f82fc9e6 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestLongValueFacet.java @@ -0,0 +1,841 @@ +/* + * 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.sandbox.facet; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.facet.LongValueFacetCounts; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.sandbox.facet.cutters.LongValueFacetCutter; +import org.apache.lucene.sandbox.facet.iterators.ComparableSupplier; +import org.apache.lucene.sandbox.facet.iterators.OrdinalIterator; +import org.apache.lucene.sandbox.facet.iterators.TopnOrdinalIterator; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.util.TestUtil; + +/** Tests long value facets, based on TestLongValueFacetCounts. */ +public class TestLongValueFacet extends SandboxFacetTestCase { + + public void testBasic() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + for (long l = 0; l < 100; l++) { + Document doc = new Document(); + doc.add(new NumericDocValuesField("field", l % 5)); + w.addDocument(doc); + } + + // Also add Long.MAX_VALUE + Document doc = new Document(); + doc.add(new NumericDocValuesField("field", Long.MAX_VALUE)); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + LongValueFacetCutter longValuesFacetCutter = new LongValueFacetCutter("field"); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + FacetResult result = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=6\n 0 (20)\n 1 (20)\n 2 (20)\n 3 (20)\n " + + "4 (20)\n 9223372036854775807 (1)\n", + result.toString()); + + FacetResult topChildrenResult = + getTopChildren(2, "field", longValuesFacetCutter, countRecorder); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=2\n 0 (20)\n 1 (20)\n", + topChildrenResult.toString()); + + assertFacetResult( + getAllChildren("field", longValuesFacetCutter, countRecorder), + "field", + new String[0], + 6, + -2147483648, + new LabelAndValue("0", 20), + new LabelAndValue("1", 20), + new LabelAndValue("2", 20), + new LabelAndValue("3", 20), + new LabelAndValue("4", 20), + new LabelAndValue("9223372036854775807", 1)); + + r.close(); + d.close(); + } + + public void testOnlyBigLongs() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + for (long l = 0; l < 3; l++) { + Document doc = new Document(); + doc.add(new NumericDocValuesField("field", Long.MAX_VALUE - l)); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + LongValueFacetCutter longValuesFacetCutter = new LongValueFacetCutter("field"); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + FacetResult result = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + + assertFacetResult( + getAllChildren("field", longValuesFacetCutter, countRecorder), + "field", + new String[0], + 3, + -2147483648, + new LabelAndValue("9223372036854775805", 1), + new LabelAndValue("9223372036854775806", 1), + new LabelAndValue("9223372036854775807", 1)); + + // since we have no insight into the value order in the hashMap, we sort labels by value and + // count in + // ascending order in order to compare with expected results + Arrays.sort( + result.labelValues, + Comparator.comparing((LabelAndValue a) -> a.label) + .thenComparingLong(a -> a.value.longValue())); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=3\n 9223372036854775805 (1)\n " + + "9223372036854775806 (1)\n 9223372036854775807 (1)\n", + result.toString()); + r.close(); + d.close(); + } + + public void testRandomSingleValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int docCount = atLeast(1000); + double missingChance = random().nextDouble(); + long maxValue; + if (random().nextBoolean()) { + maxValue = random().nextLong() & Long.MAX_VALUE; + } else { + maxValue = random().nextInt(1000); + } + if (VERBOSE) { + System.out.println( + "TEST: valueCount=" + + docCount + + " valueRange=-" + + maxValue + + "-" + + maxValue + + " missingChance=" + + missingChance); + } + Long[] values = new Long[docCount]; + // int missingCount = 0; + for (int i = 0; i < docCount; i++) { + Document doc = new Document(); + doc.add(new IntPoint("id", i)); + if (random().nextDouble() > missingChance) { + long value = TestUtil.nextLong(random(), -maxValue, maxValue); + doc.add(new NumericDocValuesField("field", value)); + values[i] = value; + } else { + // missingCount++; + } + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + int iters = atLeast(100); + for (int iter = 0; iter < iters; iter++) { + if (VERBOSE) { + System.out.println("\nTEST: iter=" + iter); + System.out.println(" test all docs"); + } + + // all docs + Map expected = new HashMap<>(); + int expectedChildCount = 0; + for (int i = 0; i < docCount; i++) { + if (values[i] != null) { + Integer curCount = expected.get(values[i]); + if (curCount == null) { + curCount = 0; + expectedChildCount++; + } + expected.put(values[i], curCount + 1); + } + } + + List> expectedCounts = new ArrayList<>(expected.entrySet()); + + // sort by value + expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey)); + + LongValueFacetCutter longValuesFacetCutter = new LongValueFacetCutter("field"); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + /* TODO: uncomment and adjust when LongValueFacetCutter supports value sources + if (random().nextBoolean()) { + if (VERBOSE) { + System.out.println(" use value source"); + } + + if (random().nextBoolean()) { + facetCounts = + new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc); + } else if (random().nextBoolean()) { + facetCounts = + new LongValueFacetCounts("field", MultiLongValuesSource.fromLongField("field"), fc); + } else { + facetCounts = + new LongValueFacetCounts( + "field", + MultiLongValuesSource.fromSingleValued(LongValuesSource.fromLongField("field")), + fc); + } + } else { */ + if (VERBOSE) { + System.out.println(" use doc values"); + } + + FacetResult actual = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + assertSame( + "all docs, sort facets by value", + expectedCounts, + expectedChildCount, + -2147483648, + // docCount - missingCount, + actual, + Integer.MAX_VALUE); + + // test getAllChildren + expectedCounts.sort( + Map.Entry.comparingByKey().thenComparingLong(Map.Entry::getValue)); + FacetResult allChildren = getAllChildren("field", longValuesFacetCutter, countRecorder); + // sort labels by value, count in ascending order + Arrays.sort( + allChildren.labelValues, + Comparator.comparing((LabelAndValue a) -> a.label) + .thenComparingLong(a -> a.value.longValue())); + assertSame( + "test getAllChildren", + expectedCounts, + expectedChildCount, + -2147483648, + // docCount - missingCount, + actual, + Integer.MAX_VALUE); + + // sort by count + expectedCounts.sort( + (a, b) -> { + int cmp = -Integer.compare(a.getValue(), b.getValue()); + if (cmp == 0) { + // tie break by value + cmp = Long.compare(a.getKey(), b.getKey()); + } + return cmp; + }); + int topN; + if (random().nextBoolean()) { + topN = docCount; + } else { + topN = random().nextInt(1, docCount); + } + if (VERBOSE) { + System.out.println(" topN=" + topN); + } + actual = getTopChildren(topN, "field", longValuesFacetCutter, countRecorder); + assertSame( + "all docs, sort facets by count", + expectedCounts, + Math.min(topN, expectedChildCount), + // expectedChildCount, + -2147483648, + // docCount - missingCount, + actual, + topN); + + // subset of docs + int minId = random().nextInt(docCount); + int maxId = random().nextInt(docCount); + if (minId > maxId) { + int tmp = minId; + minId = maxId; + maxId = tmp; + } + if (VERBOSE) { + System.out.println(" test id range " + minId + "-" + maxId); + } + + longValuesFacetCutter = new LongValueFacetCutter("field"); + countRecorder = new CountFacetRecorder(); + collectorManager = new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(IntPoint.newRangeQuery("id", minId, maxId), collectorManager); + // TODO: uncomment and change longValuesFacetCutter when LongValueFacetCutter supports value + // sources + // if (random().nextBoolean()) { + if (VERBOSE) { + System.out.println(" use doc values"); + } + /*} else { + if (VERBOSE) { + System.out.println(" use value source"); + } + if (random().nextBoolean()) { + facetCounts = + new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc); + } else if (random().nextBoolean()) { + facetCounts = + new LongValueFacetCounts("field", MultiLongValuesSource.fromLongField("field"), fc); + } else { + facetCounts = + new LongValueFacetCounts( + "field", + MultiLongValuesSource.fromSingleValued(LongValuesSource.fromLongField("field")), + fc); + } + }*/ + + expected = new HashMap<>(); + expectedChildCount = 0; + // int totCount = 0; + for (int i = minId; i <= maxId; i++) { + if (values[i] != null) { + // totCount++; + Integer curCount = expected.get(values[i]); + if (curCount == null) { + expectedChildCount++; + curCount = 0; + } + expected.put(values[i], curCount + 1); + } + } + expectedCounts = new ArrayList<>(expected.entrySet()); + + // sort by value + expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey)); + actual = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + assertSame( + "id " + minId + "-" + maxId + ", sort facets by value", + expectedCounts, + expectedChildCount, + -2147483648, + // totCount, + actual, + Integer.MAX_VALUE); + + // sort by count + expectedCounts.sort( + (a, b) -> { + int cmp = -Integer.compare(a.getValue(), b.getValue()); + if (cmp == 0) { + // tie break by value + cmp = Long.compare(a.getKey(), b.getKey()); + } + return cmp; + }); + if (random().nextBoolean()) { + topN = docCount; + } else { + topN = random().nextInt(1, docCount); + } + actual = getTopChildren(topN, "field", longValuesFacetCutter, countRecorder); + assertSame( + "id " + minId + "-" + maxId + ", sort facets by count", + expectedCounts, + Math.min(topN, expectedChildCount), + // expectedChildCount, + -2147483648, + // totCount, + actual, + topN); + } + r.close(); + dir.close(); + } + + public void testRandomMultiValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int docCount = atLeast(1000); + double missingChance = random().nextDouble(); + + // sometimes exercise codec optimizations when a claimed multi valued field is in fact single + // valued: + boolean allSingleValued = rarely(); + long maxValue; + + if (random().nextBoolean()) { + maxValue = random().nextLong() & Long.MAX_VALUE; + } else { + maxValue = random().nextInt(1000); + } + if (VERBOSE) { + System.out.println( + "TEST: valueCount=" + + docCount + + " valueRange=-" + + maxValue + + "-" + + maxValue + + " missingChance=" + + missingChance + + " allSingleValued=" + + allSingleValued); + } + + long[][] values = new long[docCount][]; + for (int i = 0; i < docCount; i++) { + Document doc = new Document(); + doc.add(new IntPoint("id", i)); + if (random().nextDouble() > missingChance) { + if (allSingleValued) { + values[i] = new long[1]; + } else { + values[i] = new long[TestUtil.nextInt(random(), 1, 5)]; + } + + for (int j = 0; j < values[i].length; j++) { + long value = TestUtil.nextLong(random(), -maxValue, maxValue); + values[i][j] = value; + doc.add(new SortedNumericDocValuesField("field", value)); + } + + if (VERBOSE) { + System.out.println(" doc=" + i + " values=" + Arrays.toString(values[i])); + } + + // sort values to enable duplicate detection by comparing with the previous value + Arrays.sort(values[i]); + } else { + if (VERBOSE) { + System.out.println(" doc=" + i + " missing values"); + } + } + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + int iters = atLeast(100); + for (int iter = 0; iter < iters; iter++) { + if (VERBOSE) { + System.out.println("\nTEST: iter=" + iter); + System.out.println(" test all docs"); + } + + // all docs + Map expected = new HashMap<>(); + // int expectedTotalCount = 0; + for (int i = 0; i < docCount; i++) { + if (values[i] != null && values[i].length > 0) { + // expectedTotalCount++; + setExpectedFrequencies(values[i], expected); + } + } + + List> expectedCounts = new ArrayList<>(expected.entrySet()); + int expectedChildCount = expected.size(); + + // sort by value + expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey)); + + LongValueFacetCutter longValuesFacetCutter = new LongValueFacetCutter("field"); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + if (VERBOSE) { + System.out.println(" use doc values"); + } + // TODO: uncomment and adjust when LongValueFacetCutter supports value sources + /*if (random().nextBoolean()) { + facetCounts = new LongValueFacetCounts("field", fc); + } else { + facetCounts = + new LongValueFacetCounts("field", MultiLongValuesSource.fromLongField("field"), fc); + }*/ + + FacetResult actual = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + assertSame( + "all docs, sort facets by value", + expectedCounts, + expectedChildCount, + -2147483648, + // expectedTotalCount, + actual, + Integer.MAX_VALUE); + + // test getAllChildren + expectedCounts.sort( + Map.Entry.comparingByKey().thenComparingLong(Map.Entry::getValue)); + FacetResult allChildren = getAllChildren("field", longValuesFacetCutter, countRecorder); + // sort labels by value, count in ascending order + Arrays.sort( + allChildren.labelValues, + Comparator.comparing((LabelAndValue a) -> a.label) + .thenComparingLong(a -> a.value.longValue())); + assertSame( + "test getAllChildren", + expectedCounts, + expectedChildCount, + -2147483648, + // expectedTotalCount, + actual, + Integer.MAX_VALUE); + + // sort by count + expectedCounts.sort( + (a, b) -> { + int cmp = -Integer.compare(a.getValue(), b.getValue()); + if (cmp == 0) { + // tie break by value + cmp = Long.compare(a.getKey(), b.getKey()); + } + return cmp; + }); + int topN; + if (random().nextBoolean()) { + topN = docCount; + } else { + topN = random().nextInt(1, docCount); + } + if (VERBOSE) { + System.out.println(" topN=" + topN); + } + actual = getTopChildren(topN, "field", longValuesFacetCutter, countRecorder); + assertSame( + "all docs, sort facets by count", + expectedCounts, + Math.min(topN, expectedChildCount), + // expectedChildCount, + -2147483648, + // expectedTotalCount, + actual, + topN); + + // subset of docs + int minId = random().nextInt(docCount); + int maxId = random().nextInt(docCount); + if (minId > maxId) { + int tmp = minId; + minId = maxId; + maxId = tmp; + } + if (VERBOSE) { + System.out.println(" test id range " + minId + "-" + maxId); + } + + longValuesFacetCutter = new LongValueFacetCutter("field"); + countRecorder = new CountFacetRecorder(); + collectorManager = new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(IntPoint.newRangeQuery("id", minId, maxId), collectorManager); + // TODO: uncomment and adjust when LongValueFacetCutter supports value sources + /*if (random().nextBoolean()) { + facetCounts = new LongValueFacetCounts("field", fc); + } else { + facetCounts = + new LongValueFacetCounts("field", MultiLongValuesSource.fromLongField("field"), fc); + }*/ + + expected = new HashMap<>(); + // expectedTotalCount = 0; + for (int i = minId; i <= maxId; i++) { + if (values[i] != null && values[i].length > 0) { + // expectedTotalCount++; + setExpectedFrequencies(values[i], expected); + } + } + expectedCounts = new ArrayList<>(expected.entrySet()); + expectedChildCount = expected.size(); + + // sort by value + expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey)); + actual = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + assertSame( + "id " + minId + "-" + maxId + ", sort facets by value", + expectedCounts, + expectedChildCount, + -2147483648, + // expectedTotalCount, + actual, + Integer.MAX_VALUE); + + // sort by count + expectedCounts.sort( + (a, b) -> { + int cmp = -Integer.compare(a.getValue(), b.getValue()); + if (cmp == 0) { + // tie break by value + cmp = Long.compare(a.getKey(), b.getKey()); + } + return cmp; + }); + if (random().nextBoolean()) { + topN = docCount; + } else { + topN = random().nextInt(1, docCount); + } + actual = getTopChildren(topN, "field", longValuesFacetCutter, countRecorder); + assertSame( + "id " + minId + "-" + maxId + ", sort facets by count", + expectedCounts, + Math.min(expectedChildCount, topN), + // expectedChildCount, + -2147483648, + // expectedTotalCount, + actual, + topN); + } + r.close(); + dir.close(); + } + + private void setExpectedFrequencies(long[] values, Map expected) { + long previousValue = 0; + for (int j = 0; j < values.length; j++) { + if (j == 0 || previousValue != values[j]) { + Integer curCount = expected.getOrDefault(values[j], 0); + expected.put(values[j], curCount + 1); + } + previousValue = values[j]; + } + } + + private static void assertSame( + String desc, + List> expectedCounts, + int expectedChildCount, + int expectedTotalCount, + FacetResult actual, + int topN) { + int expectedTopN = Math.min(topN, expectedCounts.size()); + if (VERBOSE) { + System.out.println(" expected topN=" + expectedTopN); + for (int i = 0; i < expectedTopN; i++) { + System.out.println( + " " + + i + + ": value=" + + expectedCounts.get(i).getKey() + + " count=" + + expectedCounts.get(i).getValue()); + } + System.out.println(" actual topN=" + actual.labelValues.length); + for (int i = 0; i < actual.labelValues.length; i++) { + System.out.println( + " " + + i + + ": value=" + + actual.labelValues[i].label + + " count=" + + actual.labelValues[i].value); + } + } + assertEquals(desc + ": topN", expectedTopN, actual.labelValues.length); + assertEquals(desc + ": childCount", expectedChildCount, actual.childCount); + assertEquals(desc + ": totCount", expectedTotalCount, actual.value.intValue()); + assertTrue(actual.labelValues.length <= topN); + + for (int i = 0; i < expectedTopN; i++) { + assertEquals( + desc + ": label[" + i + "]", + Long.toString(expectedCounts.get(i).getKey()), + actual.labelValues[i].label); + assertEquals( + desc + ": counts[" + i + "]", + expectedCounts.get(i).getValue().intValue(), + actual.labelValues[i].value.intValue()); + } + } + + /** + * LUCENE-9964: Duplicate long values in a document field should only be counted once when using + * SortedNumericDocValuesFields + */ + public void testDuplicateLongValues() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + // these two values are not unique in a document + doc.add(new SortedNumericDocValuesField("field", 42)); + doc.add(new SortedNumericDocValuesField("field", 42)); + w.addDocument(doc); + doc = new Document(); + doc.add(new SortedNumericDocValuesField("field", 43)); + doc.add(new SortedNumericDocValuesField("field", 43)); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + IndexSearcher s = newSearcher(r); + LongValueFacetCutter longValuesFacetCutter = new LongValueFacetCutter("field"); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longValuesFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + FacetResult fr = getAllChildrenSortByValue("field", longValuesFacetCutter, countRecorder); + for (LabelAndValue labelAndValue : fr.labelValues) { + assert labelAndValue.value.equals(1); + } + + assertFacetResult( + getAllChildren("field", longValuesFacetCutter, countRecorder), + "field", + new String[0], + 2, + -2147483648, + new LabelAndValue("42", 1), + new LabelAndValue("43", 1)); + + r.close(); + dir.close(); + } + + /** + * Get all results sorted by value, similar to {@link + * LongValueFacetCounts#getAllChildrenSortByValue()} + */ + private FacetResult getAllChildrenSortByValue( + String fieldName, + LongValueFacetCutter longValuesFacetCutter, + CountFacetRecorder countRecorder) + throws IOException { + int[] resultOrdinals = countRecorder.recordedOrds().toArray(); + ComparableSupplier comparableSupplier = + ComparableUtils.byLongValue(longValuesFacetCutter); + + ComparableUtils.sort(resultOrdinals, comparableSupplier); + + FacetLabel[] labels = longValuesFacetCutter.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return new FacetResult( + fieldName, + new String[0], + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } + + /** + * Get top results sorted by count with tie-break by value, similar to {@link + * LongValueFacetCounts#getTopChildren(int, String, String...)} + */ + private FacetResult getTopChildren( + int topN, + String field, + LongValueFacetCutter longValuesFacetCutter, + CountFacetRecorder countRecorder) + throws IOException { + ComparableSupplier comparableSupplier = + ComparableUtils.byCount(countRecorder, longValuesFacetCutter); + + OrdinalIterator topByCountOrds = + new TopnOrdinalIterator<>(countRecorder.recordedOrds(), comparableSupplier, topN); + + int[] resultOrdinals = topByCountOrds.toArray(); + + FacetLabel[] labels = longValuesFacetCutter.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return new FacetResult( + field, + new String[0], + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } + + /** + * Get all results in no particular order, similar to {@link + * LongValueFacetCounts#getAllChildren(String, String...)} + */ + private FacetResult getAllChildren( + String field, LongValueFacetCutter longValuesFacetCutter, CountFacetRecorder countRecorder) + throws IOException { + int[] resultOrdinals = countRecorder.recordedOrds().toArray(); + + FacetLabel[] labels = longValuesFacetCutter.getLabels(resultOrdinals); + List labelsAndValues = new ArrayList<>(labels.length); + int childCount = 0; + for (int i = 0; i < resultOrdinals.length; i++) { + int count = countRecorder.getCount(resultOrdinals[i]); + labelsAndValues.add(new LabelAndValue(labels[i].lastComponent(), count)); + childCount++; + } + // int value = countFacetRecorder.getCount(parentOrdinal); + return new FacetResult( + field, + new String[0], + VALUE_CANT_BE_COMPUTED, + labelsAndValues.toArray(new LabelAndValue[0]), + childCount); + } +} diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java new file mode 100644 index 00000000000..57c323e18a9 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java @@ -0,0 +1,1647 @@ +/* + * 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.sandbox.facet; + +import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.List; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoubleDocValuesField; +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.facet.DrillDownQuery; +import org.apache.lucene.facet.DrillSideways; +import org.apache.lucene.facet.FacetField; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.FacetsConfig; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.facet.MultiDoubleValuesSource; +import org.apache.lucene.facet.MultiLongValuesSource; +import org.apache.lucene.facet.range.DoubleRange; +import org.apache.lucene.facet.range.LongRange; +import org.apache.lucene.facet.range.Range; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.sandbox.facet.cutters.TaxonomyFacetsCutter; +import org.apache.lucene.sandbox.facet.cutters.ranges.DoubleRangeFacetCutter; +import org.apache.lucene.sandbox.facet.cutters.ranges.LongRangeFacetCutter; +import org.apache.lucene.sandbox.facet.labels.OrdToLabel; +import org.apache.lucene.sandbox.facet.labels.RangeOrdToLabel; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.search.CollectorOwner; +import org.apache.lucene.search.DoubleValues; +import org.apache.lucene.search.DoubleValuesSource; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LongValuesSource; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MultiCollectorManager; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.search.DummyTotalHitCountCollector; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.NumericUtils; + +/** + * Test sandbox facet ranges. Mostly test cases from LongRangeFacetCounts adopted for sandbox + * faceting. + */ +public class TestRangeFacet extends SandboxFacetTestCase { + + public void testBasicLong() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + for (long l = 0; l < 100; l++) { + field.setLongValue(l); + w.addDocument(doc); + } + + // Also add Long.MAX_VALUE + field.setLongValue(Long.MAX_VALUE); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + LongRange[] inputRanges = + new LongRange[] { + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true), + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (1)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + private int[] getRangeOrdinals(Range[] inputRanges) { + // Naive method to get a list of facet ordinals for range facets, + // it is used to get all range ordinals, including the ones that didn't match any docs. + int[] result = new int[inputRanges.length]; + for (int i = 0; i < inputRanges.length; i++) { + result[i] = i; + } + return result; + } + + public void testBasicLongMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // just index the same value twice each time and make sure we don't double count + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 100; l < 200; l++) { + field1.setLongValue(l); + // Make second value sometimes smaller, sometimes bigger, and sometimes equal + if (l % 3 == 0) { + field2.setLongValue(l - 100); + } else if (l % 3 == 1) { + field2.setLongValue(l + 100); + } else { + field2.setLongValue(l); + } + w.addDocument(doc); + } + + // Also add Long.MAX_VALUE + field1.setLongValue(Long.MAX_VALUE); + field2.setLongValue(Long.MAX_VALUE); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + ////////// Not overlapping ranges + LongRange[] inputRanges = + new LongRange[] { + new LongRange("110-120", 110L, true, 120L, true), + new LongRange("121-130", 121L, true, 130L, true), + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=2\n" + + " 110-120 (11)\n" + + " 121-130 (10)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + ///////// Overlapping ranges + inputRanges = + new LongRange[] { + new LongRange("110-120", 110L, true, 120L, true), + new LongRange("115-125", 115L, true, 125L, true), + }; + + valuesSource = MultiLongValuesSource.fromLongField("field"); + longRangeFacetCutter = LongRangeFacetCutter.create(valuesSource, inputRanges); + countRecorder = new CountFacetRecorder(); + + collectorManager = new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=2\n" + + " 110-120 (11)\n" + + " 115-125 (11)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + ////////// Multiple ranges (similar to original test) + inputRanges = + new LongRange[] { + new LongRange("[100-110)", 100L, true, 110L, false), + new LongRange("[100-110]", 100L, true, 110L, true), + new LongRange("(190-200)", 190L, false, 200L, false), + new LongRange("[190-200]", 190L, true, 200L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true) + }; + + valuesSource = MultiLongValuesSource.fromLongField("field"); + longRangeFacetCutter = LongRangeFacetCutter.create(valuesSource, inputRanges); + countRecorder = new CountFacetRecorder(); + + collectorManager = new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n" + + " [100-110) (10)\n" + + " [100-110] (11)\n" + + " (190-200) (9)\n" + + " [190-200] (10)\n" + + " over 1000 (1)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + public void testBasicLongMultiValuedMixedSegmentTypes() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + // write docs as two segments (50 in each). the first segment will contain a mix of single- and + // multi-value cases, while the second segment will be all single values. + for (int l = 0; l < 100; l++) { + field1.setLongValue(l); + field2.setLongValue(l); + Document doc = new Document(); + doc.add(field1); + if (l == 0) { + doc.add(field2); + } else if (l < 50) { + if (random().nextBoolean()) { + doc.add(field2); + } + } + w.addDocument(doc); + if (l == 50) { + w.commit(); + } + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + LongRange[] inputRanges = + new LongRange[] { + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true) + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + public void testLongMinMax() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + field.setLongValue(Long.MIN_VALUE); + w.addDocument(doc); + field.setLongValue(0); + w.addDocument(doc); + field.setLongValue(Long.MAX_VALUE); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + LongRange[] inputRanges = + new LongRange[] { + new LongRange("min", Long.MIN_VALUE, true, Long.MIN_VALUE, true), + new LongRange("max", Long.MAX_VALUE, true, Long.MAX_VALUE, true), + new LongRange("all0", Long.MIN_VALUE, true, Long.MAX_VALUE, true), + new LongRange("all1", Long.MIN_VALUE, false, Long.MAX_VALUE, true), + new LongRange("all2", Long.MIN_VALUE, true, Long.MAX_VALUE, false), + new LongRange("all3", Long.MIN_VALUE, false, Long.MAX_VALUE, false) + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=6\n min (1)\n max (1)\n all0 (3)\n all1 (2)\n all2 (2)\n all3 (1)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + public void testOverlappedEndStart() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + for (long l = 0; l < 100; l++) { + field.setLongValue(l); + w.addDocument(doc); + } + field.setLongValue(Long.MAX_VALUE); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + LongRange[] inputRanges = + new LongRange[] { + new LongRange("0-10", 0L, true, 10L, true), + new LongRange("10-20", 10L, true, 20L, true), + new LongRange("20-30", 20L, true, 30L, true), + new LongRange("30-40", 30L, true, 40L, true) + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=4\n 0-10 (11)\n 10-20 (11)\n 20-30 (11)\n 30-40 (11)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + public void testEmptyRangesSingleValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + for (long l = 0; l < 100; l++) { + field.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + LongRange[] inputRanges = new LongRange[0]; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=0\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + public void testEmptyRangesMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + field1.setLongValue(l); + field2.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + LongRange[] inputRanges = new LongRange[0]; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=0\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + r.close(); + d.close(); + } + + /** + * Tests single request that mixes Range and non-Range faceting, with DrillSideways and taxonomy. + */ + public void testMixedRangeAndNonRangeTaxonomy() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Directory td = newDirectory(); + DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + + for (long l = 0; l < 100; l++) { + Document doc = new Document(); + // For computing range facet counts: + doc.add(new NumericDocValuesField("field", l)); + // For drill down by numeric range: + doc.add(new LongPoint("field", l)); + + if ((l & 3) == 0) { + doc.add(new FacetField("dim", "a")); + } else { + doc.add(new FacetField("dim", "b")); + } + w.addDocument(config.build(tw, doc)); + } + + final IndexReader r = w.getReader(); + final TaxonomyReader tr = new DirectoryTaxonomyReader(tw); + + IndexSearcher s = newSearcher(r, false, false); + // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support + // timeouts whose implementation scores one window of doc IDs at a time. + s.setTimeout(null); + + if (VERBOSE) { + System.out.println("TEST: searcher=" + s); + } + + DrillSideways ds = + new DrillSideways(s, config, tr) { + @Override + protected boolean scoreSubDocsAtOnce() { + return random().nextBoolean(); + } + }; + + // Data for range facets + LongRange[] inputRanges = + new LongRange[] { + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false) + }; + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter fieldCutter = LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder fieldCountRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager fieldCollectorManager = + new FacetFieldCollectorManager<>(fieldCutter, fieldCountRecorder); + OrdToLabel fieldOrdToLabel = new RangeOrdToLabel(inputRanges); + + // Data for taxonomy facets + TaxonomyFacetsCutter dimCutter = new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, tr); + CountFacetRecorder dimCountRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager dimCollectorManager = + new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder); + + MultiCollectorManager collectorManager = + new MultiCollectorManager(fieldCollectorManager, dimCollectorManager); + + ////// First search, no drill-downs: + DrillDownQuery ddq = new DrillDownQuery(config); + ds.search(ddq, new CollectorOwner<>(collectorManager), List.of()); + + // assertEquals(100, dsr.hits.totalHits.value); + assertEquals( + "dim=dim path=[] value=-2147483648 childCount=2\n b (75)\n a (25)\n", + getTopChildrenByCount(dimCountRecorder, tr, 10, "dim").toString()); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), fieldCountRecorder, "field", fieldOrdToLabel) + .toString()); + + ////// Second search, drill down on dim=b: + fieldCountRecorder = new CountFacetRecorder(); + fieldCollectorManager = new FacetFieldCollectorManager<>(fieldCutter, fieldCountRecorder); + dimCountRecorder = new CountFacetRecorder(); + dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder); + ddq = new DrillDownQuery(config); + ddq.add("dim", "b"); + ds.search( + ddq, + new CollectorOwner<>(fieldCollectorManager), + List.of(new CollectorOwner<>(dimCollectorManager))); + + // assertEquals(75, dsr.hits.totalHits.value); + assertEquals( + "dim=dim path=[] value=-2147483648 childCount=2\n b (75)\n a (25)\n", + getTopChildrenByCount(dimCountRecorder, tr, 10, "dim").toString()); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (7)\n less than or equal to 10 (8)\n over 90 (7)\n 90 or above (8)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), fieldCountRecorder, "field", fieldOrdToLabel) + .toString()); + + ////// Third search, drill down on "less than or equal to 10": + fieldCountRecorder = new CountFacetRecorder(); + fieldCollectorManager = new FacetFieldCollectorManager<>(fieldCutter, fieldCountRecorder); + dimCountRecorder = new CountFacetRecorder(); + dimCollectorManager = new FacetFieldCollectorManager<>(dimCutter, dimCountRecorder); + ddq = new DrillDownQuery(config); + ddq.add("field", LongPoint.newRangeQuery("field", 0L, 10L)); + ds.search( + ddq, + new CollectorOwner<>(dimCollectorManager), + List.of(new CollectorOwner<>(fieldCollectorManager))); + + // assertEquals(11, dsr.hits.totalHits.value); + assertEquals( + "dim=dim path=[] value=-2147483648 childCount=2\n b (8)\n a (3)\n", + getTopChildrenByCount(dimCountRecorder, tr, 10, "dim").toString()); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), fieldCountRecorder, "field", fieldOrdToLabel) + .toString()); + + w.close(); + IOUtils.close(tw, tr, td, r, d); + } + + public void testBasicDouble() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + DoubleDocValuesField field = new DoubleDocValuesField("field", 0.0); + doc.add(field); + for (int i = 0; i < 100; i++) { + field.setDoubleValue(i); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r); + DoubleRange[] inputRanges = + new DoubleRange[] { + new DoubleRange("less than 10", 0.0, true, 10.0, false), + new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true), + new DoubleRange("over 90", 90.0, false, 100.0, false), + new DoubleRange("90 or above", 90.0, true, 100.0, false), + new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false) + }; + + MultiDoubleValuesSource valuesSource = MultiDoubleValuesSource.fromDoubleField("field"); + DoubleRangeFacetCutter doubleRangeFacetCutter = + new DoubleRangeFacetCutter(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + w.close(); + IOUtils.close(r, d); + } + + public void testBasicDoubleMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // index the same value twice and make sure we don't double count + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0); + doc.add(field1); + doc.add(field2); + for (int i = 0; i < 100; i++) { + field1.setLongValue(NumericUtils.doubleToSortableLong(i)); + field2.setLongValue(NumericUtils.doubleToSortableLong(i)); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r); + DoubleRange[] inputRanges = + new DoubleRange[] { + new DoubleRange("less than 10", 0.0, true, 10.0, false), + new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true), + new DoubleRange("over 90", 90.0, false, 100.0, false), + new DoubleRange("90 or above", 90.0, true, 100.0, false), + new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false) + }; + + MultiDoubleValuesSource valuesSource = MultiDoubleValuesSource.fromDoubleField("field"); + DoubleRangeFacetCutter doubleRangeFacetCutter = + new DoubleRangeFacetCutter(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + w.close(); + IOUtils.close(r, d); + } + + public void testBasicDoubleMultiValuedMixedSegmentTypes() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + // write docs as two segments (50 in each). the first segment will contain a mix of single- and + // multi-value cases, while the second segment will be all single values. + for (int l = 0; l < 100; l++) { + field1.setLongValue(NumericUtils.doubleToSortableLong(l)); + field2.setLongValue(NumericUtils.doubleToSortableLong(l)); + Document doc = new Document(); + doc.add(field1); + if (l == 0) { + doc.add(field2); + } else if (l < 50) { + if (random().nextBoolean()) { + doc.add(field2); + } + } + w.addDocument(doc); + if (l == 50) { + w.commit(); + } + } + + IndexReader r = w.getReader(); + w.close(); + + IndexSearcher s = newSearcher(r); + + DoubleRange[] inputRanges = + new DoubleRange[] { + new DoubleRange("less than 10", 0.0, true, 10.0, false), + new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true), + new DoubleRange("over 90", 90.0, false, 100.0, false), + new DoubleRange("90 or above", 90.0, true, 100.0, false), + new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false) + }; + + MultiDoubleValuesSource valuesSource = MultiDoubleValuesSource.fromDoubleField("field"); + DoubleRangeFacetCutter doubleRangeFacetCutter = + new DoubleRangeFacetCutter(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + r.close(); + d.close(); + } + + public void testRandomLongsSingleValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + if (VERBOSE) { + System.out.println("TEST: numDocs=" + numDocs); + } + long[] values = new long[numDocs]; + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + long v = random().nextLong(); + values[i] = v; + doc.add(new NumericDocValuesField("field", v)); + doc.add(new LongPoint("field", v)); + w.addDocument(doc); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 100); + LongRange[] ranges = new LongRange[numRange]; + int[] expectedCounts = new int[numRange]; + long minAcceptedValue = Long.MAX_VALUE; + long maxAcceptedValue = Long.MIN_VALUE; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + long min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextLong(); + } + long max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextLong(); + } + + if (min > max) { + long x = min; + min = max; + max = x; + } + boolean minIncl; + boolean maxIncl; + + // NOTE: max - min >= 0 is here to handle the common overflow case! + if (max - min >= 0 && max - min < 2) { + // If max == min or max == min+1, we always do inclusive, else we might pass an empty + // range and hit exc from LongRange's ctor: + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new LongRange("r" + rangeID, min, minIncl, max, maxIncl); + if (VERBOSE) { + System.out.println(" range " + rangeID + ": " + ranges[rangeID]); + } + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + boolean accept = true; + if (minIncl) { + accept &= values[i] >= min; + } else { + accept &= values[i] > min; + } + if (maxIncl) { + accept &= values[i] <= max; + } else { + accept &= values[i] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i]); + } + } + } + + // TODO: fastMatchQuery functionality is not implemented for sandbox faceting yet, do we need + // it? + /*Query fastMatchQuery; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchQuery = LongPoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchQuery = LongPoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchQuery = null; + }*/ + + final MultiLongValuesSource mvs; + if (random().nextBoolean()) { + LongValuesSource vs = LongValuesSource.fromLongField("field"); + mvs = MultiLongValuesSource.fromSingleValued(vs); + } else { + mvs = MultiLongValuesSource.fromLongField("field"); + } + + LongRangeFacetCutter longRangeFacetCutter = LongRangeFacetCutter.create(mvs, ranges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + OrdToLabel ordToLabel = new RangeOrdToLabel(ranges); + FacetResult result = + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + LongRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + ddq.add("field", LongPoint.newRangeQuery("field", range.min, range.max)); + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + + public void testRandomLongsMultiValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + if (VERBOSE) { + System.out.println("TEST: numDocs=" + numDocs); + } + long[][] values = new long[numDocs][]; + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + int numVals = RandomNumbers.randomIntBetween(random(), 1, 50); + if (random().nextInt(10) == 0) { + numVals = 1; // make sure we have ample testing of single-value cases + } + values[i] = new long[numVals]; + for (int j = 0; j < numVals; j++) { + long v = random().nextLong(); + values[i][j] = v; + doc.add(new SortedNumericDocValuesField("field", v)); + doc.add(new LongPoint("field", v)); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + w.addDocument(doc); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 100); + LongRange[] ranges = new LongRange[numRange]; + int[] expectedCounts = new int[numRange]; + long minAcceptedValue = Long.MAX_VALUE; + long maxAcceptedValue = Long.MIN_VALUE; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + long min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextLong(); + } + long max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextLong(); + } + + if (min > max) { + long x = min; + min = max; + max = x; + } + boolean minIncl; + boolean maxIncl; + + // NOTE: max - min >= 0 is here to handle the common overflow case! + if (max - min >= 0 && max - min < 2) { + // If max == min or max == min+1, we always do inclusive, else we might pass an empty + // range and hit exc from LongRange's ctor: + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new LongRange("r" + rangeID, min, minIncl, max, maxIncl); + if (VERBOSE) { + System.out.println(" range " + rangeID + ": " + ranges[rangeID]); + } + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + for (int j = 0; j < values[i].length; j++) { + boolean accept = true; + if (minIncl) { + accept &= values[i][j] >= min; + } else { + accept &= values[i][j] > min; + } + if (maxIncl) { + accept &= values[i][j] <= max; + } else { + accept &= values[i][j] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i][j]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i][j]); + break; // ensure each doc can contribute at most 1 count to each range + } + } + } + } + + // TODO: fastMatchQuery functionality is not implemented for sandbox faceting yet, do we need + // it? + /*Query fastMatchQuery; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchQuery = LongPoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchQuery = LongPoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchQuery = null; + }*/ + final MultiLongValuesSource mvs = MultiLongValuesSource.fromLongField("field"); + + LongRangeFacetCutter longRangeFacetCutter = LongRangeFacetCutter.create(mvs, ranges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + OrdToLabel ordToLabel = new RangeOrdToLabel(ranges); + FacetResult result = + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + LongRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + if (random().nextBoolean()) { + ddq.add("field", LongPoint.newRangeQuery("field", range.min, range.max)); + } else { + ddq.add( + "field", + SortedNumericDocValuesField.newSlowRangeQuery("field", range.min, range.max)); + } + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + + public void testRandomDoublesSingleValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + double[] values = new double[numDocs]; + double minValue = Double.POSITIVE_INFINITY; + double maxValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + double v = random().nextDouble(); + values[i] = v; + doc.add(new DoubleDocValuesField("field", v)); + doc.add(new DoublePoint("field", v)); + w.addDocument(doc); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 5); + DoubleRange[] ranges = new DoubleRange[numRange]; + int[] expectedCounts = new int[numRange]; + double minAcceptedValue = Double.POSITIVE_INFINITY; + double maxAcceptedValue = Double.NEGATIVE_INFINITY; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + double min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextDouble(); + } + double max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextDouble(); + } + + if (min > max) { + double x = min; + min = max; + max = x; + } + + boolean minIncl; + boolean maxIncl; + + long minAsLong = NumericUtils.doubleToSortableLong(min); + long maxAsLong = NumericUtils.doubleToSortableLong(max); + // NOTE: maxAsLong - minAsLong >= 0 is here to handle the common overflow case! + if (maxAsLong - minAsLong >= 0 && maxAsLong - minAsLong < 2) { + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new DoubleRange("r" + rangeID, min, minIncl, max, maxIncl); + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + boolean accept = true; + if (minIncl) { + accept &= values[i] >= min; + } else { + accept &= values[i] > min; + } + if (maxIncl) { + accept &= values[i] <= max; + } else { + accept &= values[i] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i]); + } + } + } + + // TODO: fastMatchQuery functionality is not implemented for sandbox faceting yet, do we need + // it? + /*Query fastMatchFilter; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchFilter = DoublePoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchFilter = DoublePoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchFilter = null; + }*/ + + final MultiDoubleValuesSource mvs; + if (random().nextBoolean()) { + DoubleValuesSource vs = DoubleValuesSource.fromDoubleField("field"); + mvs = MultiDoubleValuesSource.fromSingleValued(vs); + } else { + mvs = MultiDoubleValuesSource.fromDoubleField("field"); + } + + DoubleRangeFacetCutter doubleRangeFacetCutter = new DoubleRangeFacetCutter(mvs, ranges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + OrdToLabel ordToLabel = new RangeOrdToLabel(ranges); + FacetResult result = + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + DoubleRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + ddq.add("field", DoublePoint.newRangeQuery("field", range.min, range.max)); + + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + + public void testRandomDoublesMultiValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + double[][] values = new double[numDocs][]; + double minValue = Double.POSITIVE_INFINITY; + double maxValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + int numVals = RandomNumbers.randomIntBetween(random(), 1, 50); + if (random().nextInt(10) == 0) { + numVals = 1; // make sure we have ample testing of single-value cases + } + values[i] = new double[numVals]; + for (int j = 0; j < numVals; j++) { + double v = random().nextDouble(); + values[i][j] = v; + doc.add(new SortedNumericDocValuesField("field", Double.doubleToLongBits(v))); + doc.add(new DoublePoint("field", v)); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + w.addDocument(doc); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 5); + DoubleRange[] ranges = new DoubleRange[numRange]; + int[] expectedCounts = new int[numRange]; + double minAcceptedValue = Double.POSITIVE_INFINITY; + double maxAcceptedValue = Double.NEGATIVE_INFINITY; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + double min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextDouble(); + } + double max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextDouble(); + } + + if (min > max) { + double x = min; + min = max; + max = x; + } + + boolean minIncl; + boolean maxIncl; + + long minAsLong = NumericUtils.doubleToSortableLong(min); + long maxAsLong = NumericUtils.doubleToSortableLong(max); + // NOTE: maxAsLong - minAsLong >= 0 is here to handle the common overflow case! + if (maxAsLong - minAsLong >= 0 && maxAsLong - minAsLong < 2) { + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new DoubleRange("r" + rangeID, min, minIncl, max, maxIncl); + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + for (int j = 0; j < values[i].length; j++) { + boolean accept = true; + if (minIncl) { + accept &= values[i][j] >= min; + } else { + accept &= values[i][j] > min; + } + if (maxIncl) { + accept &= values[i][j] <= max; + } else { + accept &= values[i][j] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i][j]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i][j]); + break; // ensure each doc can contribute at most 1 count to each range + } + } + } + } + // TODO: fastMatchQuery functionality is not implemented for sandbox faceting yet, do we need + // it? + /*Query fastMatchFilter; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchFilter = DoublePoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchFilter = DoublePoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchFilter = null; + }*/ + final MultiDoubleValuesSource mvs = MultiDoubleValuesSource.fromDoubleField("field"); + DoubleRangeFacetCutter doubleRangeFacetCutter = new DoubleRangeFacetCutter(mvs, ranges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + + OrdToLabel ordToLabel = new RangeOrdToLabel(ranges); + FacetResult result = + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + DoubleRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + if (random().nextBoolean()) { + ddq.add("field", DoublePoint.newRangeQuery("field", range.min, range.max)); + } else { + ddq.add( + "field", + SortedNumericDocValuesField.newSlowRangeQuery( + "field", Double.doubleToLongBits(range.min), Double.doubleToLongBits(range.max))); + } + + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + + // LUCENE-5178 + public void testMissingValues() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + for (long l = 0; l < 100; l++) { + if (l % 5 == 0) { + // Every 5th doc is missing the value: + w.addDocument(new Document()); + continue; + } + field.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r); + LongRange[] inputRanges = + new LongRange[] { + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false) + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (8)\n less than or equal to 10 (8)\n over 90 (8)\n 90 or above (8)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + w.close(); + IOUtils.close(r, d); + } + + public void testMissingValuesMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // index the same field twice to test multi-valued logic + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + if (l % 5 == 0) { + // Every 5th doc is missing the value: + w.addDocument(new Document()); + continue; + } + field1.setLongValue(l); + field2.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r); + LongRange[] inputRanges = + new LongRange[] { + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false) + }; + + MultiLongValuesSource valuesSource = MultiLongValuesSource.fromLongField("field"); + LongRangeFacetCutter longRangeFacetCutter = + LongRangeFacetCutter.create(valuesSource, inputRanges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(longRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(inputRanges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=5\n less than 10 (8)\n less than or equal to 10 (8)\n over 90 (8)\n 90 or above (8)\n over 1000 (0)\n", + getAllSortByOrd(getRangeOrdinals(inputRanges), countRecorder, "field", ordToLabel) + .toString()); + + w.close(); + IOUtils.close(r, d); + } + + private static class PlusOneValuesSource extends DoubleValuesSource { + + @Override + public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException { + return new DoubleValues() { + int doc = -1; + + @Override + public double doubleValue() throws IOException { + return doc + 1; + } + + @Override + public boolean advanceExact(int doc) throws IOException { + this.doc = doc; + return true; + } + }; + } + + @Override + public boolean needsScores() { + return false; + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return false; + } + + @Override + public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) + throws IOException { + return Explanation.match(docId + 1, ""); + } + + @Override + public DoubleValuesSource rewrite(IndexSearcher searcher) throws IOException { + return this; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean equals(Object obj) { + return obj.getClass() == PlusOneValuesSource.class; + } + + @Override + public String toString() { + return null; + } + } + + public void testCustomDoubleValuesSource() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + writer.addDocument(doc); + writer.addDocument(doc); + writer.addDocument(doc); + + // Test wants 3 docs in one segment: + writer.forceMerge(1); + + final DoubleValuesSource vs = new PlusOneValuesSource(); + + FacetsConfig config = new FacetsConfig(); + + IndexReader r = writer.getReader(); + + IndexSearcher s = newSearcher(r, false, false); + // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support + // timeouts whose implementation scores one window of doc IDs at a time. + s.setTimeout(null); + + final DoubleRange[] ranges = + new DoubleRange[] { + new DoubleRange("< 1", 0.0, true, 1.0, false), + new DoubleRange("< 2", 0.0, true, 2.0, false), + new DoubleRange("< 5", 0.0, true, 5.0, false), + new DoubleRange("< 10", 0.0, true, 10.0, false), + new DoubleRange("< 20", 0.0, true, 20.0, false), + new DoubleRange("< 50", 0.0, true, 50.0, false) + }; + + // TODO: fastMatchQuery functionality is not implemented for sandbox faceting yet, do we need + // it? + /*final Query fastMatchFilter; + final AtomicBoolean filterWasUsed = new AtomicBoolean(); + if (random().nextBoolean()) { + // Sort of silly: + final Query in = new MatchAllDocsQuery(); + fastMatchFilter = new UsedQuery(in, filterWasUsed); + } else { + fastMatchFilter = null; + } + + if (VERBOSE) { + System.out.println("TEST: fastMatchFilter=" + fastMatchFilter); + }*/ + + MultiDoubleValuesSource valuesSource = MultiDoubleValuesSource.fromSingleValued(vs); + DoubleRangeFacetCutter doubleRangeFacetCutter = + new DoubleRangeFacetCutter(valuesSource, ranges); + CountFacetRecorder countRecorder = new CountFacetRecorder(); + + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder); + s.search(new MatchAllDocsQuery(), collectorManager); + OrdToLabel ordToLabel = new RangeOrdToLabel(ranges); + + assertEquals( + "dim=field path=[] value=-2147483648 childCount=6\n < 1 (0)\n < 2 (1)\n < 5 (3)\n < 10 (3)\n < 20 (3)\n < 50 (3)\n", + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel).toString()); + // assertTrue(fastMatchFilter == null || filterWasUsed.get()); + + DrillDownQuery ddq = new DrillDownQuery(config); + if (random().nextBoolean()) { + ddq.add("field", ranges[1].getQuery(null, vs)); + } else { + ddq.add("field", ranges[1].getQuery(null, MultiDoubleValuesSource.fromSingleValued(vs))); + } + + // Test simple drill-down: + assertEquals(1, s.search(ddq, 10).totalHits.value); + + // Test drill-sideways after drill-down + DrillSideways ds = + new DrillSideways(s, config, (TaxonomyReader) null) { + @Override + protected boolean scoreSubDocsAtOnce() { + return random().nextBoolean(); + } + }; + + countRecorder = new CountFacetRecorder(); + + CollectorOwner totalHitsCollectorOwner = + new CollectorOwner<>(DummyTotalHitCountCollector.createManager()); + CollectorOwner drillSidewaysCollectorOwner = + new CollectorOwner<>( + new FacetFieldCollectorManager<>(doubleRangeFacetCutter, countRecorder)); + ds.search(ddq, totalHitsCollectorOwner, List.of(drillSidewaysCollectorOwner)); + assertEquals(1, totalHitsCollectorOwner.getResult().intValue()); + drillSidewaysCollectorOwner.getResult(); + assertEquals( + "dim=field path=[] value=-2147483648 childCount=6\n < 1 (0)\n < 2 (1)\n < 5 (3)\n < 10 (3)\n < 20 (3)\n < 50 (3)\n", + getAllSortByOrd(getRangeOrdinals(ranges), countRecorder, "field", ordToLabel).toString()); + + writer.close(); + IOUtils.close(r, dir); + } +} diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestTaxonomyFacet.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestTaxonomyFacet.java new file mode 100644 index 00000000000..fda47497498 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestTaxonomyFacet.java @@ -0,0 +1,211 @@ +/* + * 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.sandbox.facet; + +import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME; + +import org.apache.lucene.document.Document; +import org.apache.lucene.facet.DrillDownQuery; +import org.apache.lucene.facet.FacetField; +import org.apache.lucene.facet.FacetsConfig; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.facet.taxonomy.FacetLabel; +import org.apache.lucene.facet.taxonomy.TaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader; +import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.sandbox.facet.cutters.TaxonomyFacetsCutter; +import org.apache.lucene.sandbox.facet.labels.TaxonomyOrdLabelBiMap; +import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.IOUtils; + +/** Test for associations */ +public class TestTaxonomyFacet extends SandboxFacetTestCase { + + public void testConstants() { + // It is essential for TaxonomyOrdLabelBiMap that invalid ordinal is the same as for + // TaxonomyReader + assertEquals(TaxonomyOrdLabelBiMap.INVALID_ORD, TaxonomyReader.INVALID_ORDINAL); + } + + public void testBasic() throws Exception { + Directory dir = newDirectory(); + Directory taxoDir = newDirectory(); + + // Writes facet ords to a separate directory from the + // main index: + DirectoryTaxonomyWriter taxoWriter = + new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + config.setHierarchical("Publish Date", true); + + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new FacetField("Author", "Bob")); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2010", "10", "20")); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Lisa")); + doc.add(new FacetField("Publish Date", "2012", "1", "1")); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Susan")); + doc.add(new FacetField("Publish Date", "2012", "1", "7")); + writer.addDocument(config.build(taxoWriter, doc)); + + doc = new Document(); + doc.add(new FacetField("Author", "Frank")); + doc.add(new FacetField("Publish Date", "1999", "5", "5")); + writer.addDocument(config.build(taxoWriter, doc)); + + // NRT open + IndexSearcher searcher = newSearcher(writer.getReader()); + + // NRT open + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter); + + Query query = new MatchAllDocsQuery(); + + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader); + final CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, countRecorder); + searcher.search(query, collectorManager); + + expectThrows( + IllegalArgumentException.class, + () -> { + getTopChildrenByCount(countRecorder, taxoReader, 0, "Author"); + }); + + // Retrieve & verify results: + assertEquals( + "dim=Publish Date path=[] value=-2147483648 childCount=3\n 2010 (2)\n 2012 (2)\n 1999 (1)\n", + getTopChildrenByCount(countRecorder, taxoReader, 10, "Publish Date").toString()); + assertEquals( + "dim=Author path=[] value=-2147483648 childCount=4\n Lisa (2)\n Bob (1)\n Susan (1)\n Frank (1)\n", + getTopChildrenByCount(countRecorder, taxoReader, 10, "Author").toString()); + + assertFacetResult( + getAllChildren(countRecorder, taxoReader, "Publish Date"), + "Publish Date", + new String[0], + 3, + VALUE_CANT_BE_COMPUTED, + new LabelAndValue[] { + new LabelAndValue("1999", 1), new LabelAndValue("2010", 2), new LabelAndValue("2012", 2), + }); + + assertFacetResult( + getAllChildren(countRecorder, taxoReader, "Author"), + "Author", + new String[0], + 4, + VALUE_CANT_BE_COMPUTED, + new LabelAndValue[] { + new LabelAndValue("Bob", 1), + new LabelAndValue("Frank", 1), + new LabelAndValue("Lisa", 2), + new LabelAndValue("Susan", 1), + }); + + // Now user drills down on Publish Date/2010: + DrillDownQuery q2 = new DrillDownQuery(config); + q2.add("Publish Date", "2010"); + final CountFacetRecorder countRecorder2 = new CountFacetRecorder(); + collectorManager = new FacetFieldCollectorManager<>(defaultTaxoCutter, countRecorder2); + searcher.search(q2, collectorManager); + + assertEquals( + "dim=Author path=[] value=-2147483648 childCount=2\n Bob (1)\n Lisa (1)\n", + getTopChildrenByCount(countRecorder2, taxoReader, 10, "Author").toString()); + + assertEquals(1, getSpecificValue(countRecorder2, taxoReader, "Author", "Lisa")); + + assertArrayEquals( + new int[] {1, 1}, + getCountsForRecordedCandidates( + countRecorder2, + taxoReader, + new FacetLabel[] { + new FacetLabel("Author", "Lisa"), + new FacetLabel("Author", "Susan"), // 0 count, filtered out + new FacetLabel("Author", "DoesNotExist"), // Doesn't exist in the index, filtered out + new FacetLabel("Author", "Bob"), + })); + + expectThrows( + AssertionError.class, + () -> { + getTopChildrenByCount(countRecorder2, taxoReader, 10, "Non exitent dim"); + }); + + writer.close(); + IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, taxoDir, dir); + } + + public void testTaxonomyCutterExpertModeDisableRollup() throws Exception { + Directory dir = newDirectory(); + Directory taxoDir = newDirectory(); + + DirectoryTaxonomyWriter taxoWriter = + new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE); + + FacetsConfig config = new FacetsConfig(); + config.setHierarchical("Publish Date", true); + + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new FacetField("Publish Date", "2010", "10", "15")); + writer.addDocument(config.build(taxoWriter, doc)); + + IndexSearcher searcher = newSearcher(writer.getReader()); + TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter); + Query query = new MatchAllDocsQuery(); + + TaxonomyFacetsCutter defaultTaxoCutter = + new TaxonomyFacetsCutter(DEFAULT_INDEX_FIELD_NAME, config, taxoReader, true); + final CountFacetRecorder countRecorder = new CountFacetRecorder(); + FacetFieldCollectorManager collectorManager = + new FacetFieldCollectorManager<>(defaultTaxoCutter, countRecorder); + searcher.search(query, collectorManager); + + assertEquals( + "Only leaf value should have been counted when rollup is disabled", + 1, + countRecorder.recordedOrds().toArray().length); + + writer.close(); + IOUtils.close(taxoWriter, searcher.getIndexReader(), taxoReader, taxoDir, dir); + } +} From beea8777757889cd34fe74bfcfd21a442c06134a Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Mon, 12 Aug 2024 07:30:36 -0700 Subject: [PATCH 63/66] Move CHANGES entry for GH#13568 --- lucene/CHANGES.txt | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f78ec8ce02f..f49685ffd25 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -108,15 +108,6 @@ API Changes * GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand) -* GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and - MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar) - -* GITHUB#13568: Add CollectorOwner class that wraps CollectorManager, and handles list of Collectors and results. - Add IndexSearcher#search method that takes CollectorOwner. (Egor Potemkin) - -* GITHUB#13568: Add DrillSideways#search method that supports any collector types for any drill-sideways dimensions - or drill-down. (Egor Potemkin) - * GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski) * GITHUB#13632: CandidateMatcher public matching functions (Bryan Jacobowitz) @@ -143,8 +134,6 @@ New Features DocValuesSkipper abstraction. A new flag is added to FieldType.java that configures whether to create a "skip index" for doc values. (Ignacio Vera) -* GITHUB#13568: Add sandbox facets module to compute facets while collecting. (Egor Potemkin, Shradha Shankar) - * GITHUB#13563: Add levels to doc values skip index. (Ignacio Vera) * GITHUB#13597: Align doc value skipper interval boundaries when an interval contains a constant @@ -152,7 +141,6 @@ New Features * GITHUB#13604: Add Kmeans clustering on vectors (Mayya Sharipova, Jim Ferenczi, Tom Veasey) - Improvements --------------------- @@ -280,6 +268,15 @@ API Changes * GITHUB#13559: Add BitSet#nextSetBit(int, int) to get the index of the first set bit in range. (Egor Potemkin) +* GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and + MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar) + +* GITHUB#13568: Add CollectorOwner class that wraps CollectorManager, and handles list of Collectors and results. + Add IndexSearcher#search method that takes CollectorOwner. (Egor Potemkin) + +* GITHUB#13568: Add DrillSideways#search method that supports any collector types for any drill-sideways dimensions + or drill-down. (Egor Potemkin) + New Features --------------------- @@ -292,6 +289,8 @@ New Features and LogByteSizeMergePolicy via a new #setTargetConcurrency setter. (Adrien Grand) +* GITHUB#13568: Add sandbox facets module to compute facets while collecting. (Egor Potemkin, Shradha Shankar) + Improvements --------------------- From a40bb39195467abdad0c612ee0d006c67cc75914 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 13 Aug 2024 21:49:10 +0200 Subject: [PATCH 64/66] Optimize decoding blocks of postings using the vector API. (#13636) Our postings use a layout that helps take advantage of Java's auto-vectorization to be reasonably fast to decode. But we can make it a bit faster by vectorizing directly from the MemorySegment instead of first copying data into a long[]. This approach only works when the `Directory` uses `MemorySegmentIndexInput` under the hood, ie. `MMapDirectory` on JDK 21+. Co-authored-by: Uwe Schindler --- lucene/CHANGES.txt | 3 + .../jmh/PostingIndexInputBenchmark.java | 106 +++++ .../generated/checksums/generateForUtil.json | 4 +- .../lucene/codecs/lucene912/ForDeltaUtil.java | 8 +- .../lucene/codecs/lucene912/ForUtil.java | 445 +++++++++--------- .../lucene912/Lucene912PostingsReader.java | 63 ++- .../lucene912/Lucene912PostingsWriter.java | 4 +- .../lucene/codecs/lucene912/PForUtil.java | 12 +- .../codecs/lucene912/PostingIndexInput.java | 55 +++ .../lucene/codecs/lucene912/gen_ForUtil.py | 104 ++-- .../DefaultPostingDecodingUtil.java | 42 ++ .../DefaultVectorizationProvider.java | 6 + .../vectorization/PostingDecodingUtil.java | 32 ++ .../vectorization/VectorizationProvider.java | 8 +- .../MemorySegmentPostingDecodingUtil.java | 85 ++++ .../vectorization/PanamaVectorConstants.java | 55 +++ .../PanamaVectorUtilSupport.java | 26 +- .../PanamaVectorizationProvider.java | 32 +- .../store/MemorySegmentAccessInput.java | 2 +- .../lucene/store/MemorySegmentIndexInput.java | 5 +- .../codecs/lucene912/TestForDeltaUtil.java | 6 +- .../lucene/codecs/lucene912/TestForUtil.java | 5 +- .../lucene/codecs/lucene912/TestPForUtil.java | 6 +- .../TestPostingDecodingUtil.java | 81 ++++ 24 files changed, 849 insertions(+), 346 deletions(-) create mode 100644 lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java create mode 100644 lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultPostingDecodingUtil.java create mode 100644 lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java create mode 100644 lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java create mode 100644 lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorConstants.java create mode 100644 lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f49685ffd25..49a4da82b53 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -352,6 +352,9 @@ Optimizations * GITHUB#13581: OnHeapHnswGraph no longer allocates a lock for every graph node (Mike Sokolov) +* GITHUB#13636: Optimizations to the decoding logic of blocks of postings. + (Adrien Grand, Uwe Schindler) + Changes in runtime behavior --------------------- diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java new file mode 100644 index 00000000000..3804af0b167 --- /dev/null +++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java @@ -0,0 +1,106 @@ +/* + * 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.benchmark.jmh; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import org.apache.lucene.codecs.lucene912.ForUtil; +import org.apache.lucene.codecs.lucene912.PostingIndexInput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.MMapDirectory; +import org.apache.lucene.util.IOUtils; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Benchmark) +@Warmup(iterations = 5, time = 1) +@Measurement(iterations = 5, time = 1) +@Fork( + value = 3, + jvmArgsAppend = {"-Xmx1g", "-Xms1g", "-XX:+AlwaysPreTouch"}) +public class PostingIndexInputBenchmark { + + private Path path; + private Directory dir; + private IndexInput in; + private PostingIndexInput postingIn; + private final ForUtil forUtil = new ForUtil(); + private final long[] values = new long[128]; + + @Param({"5", "6", "7", "8", "9", "10"}) + public int bpv; + + @Setup(Level.Trial) + public void setup() throws Exception { + path = Files.createTempDirectory("forUtil"); + dir = MMapDirectory.open(path); + try (IndexOutput out = dir.createOutput("docs", IOContext.DEFAULT)) { + Random r = new Random(0); + // Write enough random data to not reach EOF while decoding + for (int i = 0; i < 100; ++i) { + out.writeLong(r.nextLong()); + } + } + in = dir.openInput("docs", IOContext.DEFAULT); + postingIn = new PostingIndexInput(in, forUtil); + } + + @TearDown(Level.Trial) + public void tearDown() throws Exception { + if (dir != null) { + dir.deleteFile("docs"); + } + IOUtils.close(in, dir); + in = null; + dir = null; + Files.deleteIfExists(path); + } + + @Benchmark + public void decode(Blackhole bh) throws IOException { + in.seek(3); // random unaligned offset + postingIn.decode(bpv, values); + bh.consume(values); + } + + @Benchmark + public void decodeAndPrefixSum(Blackhole bh) throws IOException { + in.seek(3); // random unaligned offset + postingIn.decodeAndPrefixSum(bpv, 100, values); + bh.consume(values); + } +} diff --git a/lucene/core/src/generated/checksums/generateForUtil.json b/lucene/core/src/generated/checksums/generateForUtil.json index 752285f4d7f..e147f2c62f7 100644 --- a/lucene/core/src/generated/checksums/generateForUtil.json +++ b/lucene/core/src/generated/checksums/generateForUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "5ff856e80cab30f9e5704aa89f3197f017d07624", - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "3ccf92b3ddbff6340a13e8a55090bfb900dc7be2" + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "cc7d40997e2d6500b79c19ff47461ed6e89d2268", + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "ba029f2e374e66c6cf315b2c93f4efa6944dfbb8" } \ No newline at end of file diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java index 8b9aedcfb2b..b53c18fa3f8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java @@ -22,7 +22,7 @@ import org.apache.lucene.store.DataOutput; import org.apache.lucene.util.packed.PackedInts; /** Utility class to encode/decode increasing sequences of 128 integers. */ -public class ForDeltaUtil { +final class ForDeltaUtil { // IDENTITY_PLUS_ONE[i] == i+1 private static final long[] IDENTITY_PLUS_ONE = new long[ForUtil.BLOCK_SIZE]; @@ -67,12 +67,12 @@ public class ForDeltaUtil { } /** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */ - void decodeAndPrefixSum(DataInput in, long base, long[] longs) throws IOException { - final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); + void decodeAndPrefixSum(PostingIndexInput in, long base, long[] longs) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.in.readByte()); if (bitsPerValue == 0) { prefixSumOfOnes(longs, base); } else { - forUtil.decodeAndPrefixSum(bitsPerValue, in, base, longs); + in.decodeAndPrefixSum(bitsPerValue, base, longs); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java index 63ee7baaf10..4408a8a3e57 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java @@ -19,17 +19,18 @@ package org.apache.lucene.codecs.lucene912; import java.io.IOException; -import org.apache.lucene.store.DataInput; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; -// Inspired from https://fulmicoton.com/posts/bitpacking/ -// Encodes multiple integers in a long to get SIMD-like speedups. -// If bitsPerValue <= 8 then we pack 8 ints per long -// else if bitsPerValue <= 16 we pack 4 ints per long -// else we pack 2 ints per long -final class ForUtil { +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ Encodes multiple integers in a long to get + * SIMD-like speedups. If bitsPerValue <= 8 then we pack 8 ints per long else if bitsPerValue + * <= 16 we pack 4 ints per long else we pack 2 ints per long + */ +public final class ForUtil { - static final int BLOCK_SIZE = 128; + public static final int BLOCK_SIZE = 128; private static final int BLOCK_SIZE_LOG2 = 7; private static long expandMask32(long mask32) { @@ -300,13 +301,14 @@ final class ForUtil { return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); } - private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + private static void decodeSlow( + int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { final int numLongs = bitsPerValue << 1; - in.readLongs(tmp, 0, numLongs); final long mask = MASKS32[bitsPerValue]; - int longsIdx = 0; - int shift = 32 - bitsPerValue; + pdu.splitLongs(numLongs, longs, 32 - bitsPerValue, mask, tmp, 0, -1L); + int longsIdx = numLongs; + int shift = 32 - 2 * bitsPerValue; for (; shift >= 0; shift -= bitsPerValue) { shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); longsIdx += numLongs; @@ -342,6 +344,13 @@ final class ForUtil { } } + /** Likewise, but for a simple mask. */ + private static void maskLongs(long[] a, int count, long[] b, int bi, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = a[i] & mask; + } + } + private static final long[] MASKS8 = new long[8]; private static final long[] MASKS16 = new long[16]; private static final long[] MASKS32 = new long[32]; @@ -406,279 +415,280 @@ final class ForUtil { private static final long MASK32_24 = MASKS32[24]; /** Decode 128 integers into {@code longs}. */ - void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + void decode(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] longs) + throws IOException { switch (bitsPerValue) { case 1: - decode1(in, tmp, longs); + decode1(in, pdu, tmp, longs); expand8(longs); break; case 2: - decode2(in, tmp, longs); + decode2(in, pdu, tmp, longs); expand8(longs); break; case 3: - decode3(in, tmp, longs); + decode3(in, pdu, tmp, longs); expand8(longs); break; case 4: - decode4(in, tmp, longs); + decode4(in, pdu, tmp, longs); expand8(longs); break; case 5: - decode5(in, tmp, longs); + decode5(in, pdu, tmp, longs); expand8(longs); break; case 6: - decode6(in, tmp, longs); + decode6(in, pdu, tmp, longs); expand8(longs); break; case 7: - decode7(in, tmp, longs); + decode7(in, pdu, tmp, longs); expand8(longs); break; case 8: - decode8(in, tmp, longs); + decode8(in, pdu, tmp, longs); expand8(longs); break; case 9: - decode9(in, tmp, longs); + decode9(in, pdu, tmp, longs); expand16(longs); break; case 10: - decode10(in, tmp, longs); + decode10(in, pdu, tmp, longs); expand16(longs); break; case 11: - decode11(in, tmp, longs); + decode11(in, pdu, tmp, longs); expand16(longs); break; case 12: - decode12(in, tmp, longs); + decode12(in, pdu, tmp, longs); expand16(longs); break; case 13: - decode13(in, tmp, longs); + decode13(in, pdu, tmp, longs); expand16(longs); break; case 14: - decode14(in, tmp, longs); + decode14(in, pdu, tmp, longs); expand16(longs); break; case 15: - decode15(in, tmp, longs); + decode15(in, pdu, tmp, longs); expand16(longs); break; case 16: - decode16(in, tmp, longs); + decode16(in, pdu, tmp, longs); expand16(longs); break; case 17: - decode17(in, tmp, longs); + decode17(in, pdu, tmp, longs); expand32(longs); break; case 18: - decode18(in, tmp, longs); + decode18(in, pdu, tmp, longs); expand32(longs); break; case 19: - decode19(in, tmp, longs); + decode19(in, pdu, tmp, longs); expand32(longs); break; case 20: - decode20(in, tmp, longs); + decode20(in, pdu, tmp, longs); expand32(longs); break; case 21: - decode21(in, tmp, longs); + decode21(in, pdu, tmp, longs); expand32(longs); break; case 22: - decode22(in, tmp, longs); + decode22(in, pdu, tmp, longs); expand32(longs); break; case 23: - decode23(in, tmp, longs); + decode23(in, pdu, tmp, longs); expand32(longs); break; case 24: - decode24(in, tmp, longs); + decode24(in, pdu, tmp, longs); expand32(longs); break; default: - decodeSlow(bitsPerValue, in, tmp, longs); + decodeSlow(bitsPerValue, in, pdu, tmp, longs); expand32(longs); break; } } /** Delta-decode 128 integers into {@code longs}. */ - void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) + void decodeAndPrefixSum( + int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long base, long[] longs) throws IOException { switch (bitsPerValue) { case 1: - decode1(in, tmp, longs); + decode1(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 2: - decode2(in, tmp, longs); + decode2(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 3: - decode3(in, tmp, longs); + decode3(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 4: - decode4(in, tmp, longs); + decode4(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 5: - decode5(in, tmp, longs); + decode5(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 6: - decode6(in, tmp, longs); + decode6(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 7: - decode7(in, tmp, longs); + decode7(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 8: - decode8(in, tmp, longs); + decode8(in, pdu, tmp, longs); prefixSum8(longs, base); break; case 9: - decode9(in, tmp, longs); + decode9(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 10: - decode10(in, tmp, longs); + decode10(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 11: - decode11(in, tmp, longs); + decode11(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 12: - decode12(in, tmp, longs); + decode12(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 13: - decode13(in, tmp, longs); + decode13(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 14: - decode14(in, tmp, longs); + decode14(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 15: - decode15(in, tmp, longs); + decode15(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 16: - decode16(in, tmp, longs); + decode16(in, pdu, tmp, longs); prefixSum16(longs, base); break; case 17: - decode17(in, tmp, longs); + decode17(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 18: - decode18(in, tmp, longs); + decode18(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 19: - decode19(in, tmp, longs); + decode19(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 20: - decode20(in, tmp, longs); + decode20(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 21: - decode21(in, tmp, longs); + decode21(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 22: - decode22(in, tmp, longs); + decode22(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 23: - decode23(in, tmp, longs); + decode23(in, pdu, tmp, longs); prefixSum32(longs, base); break; case 24: - decode24(in, tmp, longs); + decode24(in, pdu, tmp, longs); prefixSum32(longs, base); break; default: - decodeSlow(bitsPerValue, in, tmp, longs); + decodeSlow(bitsPerValue, in, pdu, tmp, longs); prefixSum32(longs, base); break; } } - private static void decode1(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 2); - shiftLongs(tmp, 2, longs, 0, 7, MASK8_1); + private static void decode1(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(2, longs, 7, MASK8_1, tmp, 0, MASK8_7); shiftLongs(tmp, 2, longs, 2, 6, MASK8_1); shiftLongs(tmp, 2, longs, 4, 5, MASK8_1); shiftLongs(tmp, 2, longs, 6, 4, MASK8_1); shiftLongs(tmp, 2, longs, 8, 3, MASK8_1); shiftLongs(tmp, 2, longs, 10, 2, MASK8_1); shiftLongs(tmp, 2, longs, 12, 1, MASK8_1); - shiftLongs(tmp, 2, longs, 14, 0, MASK8_1); + maskLongs(tmp, 2, longs, 14, MASK8_1); } - private static void decode2(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 4); - shiftLongs(tmp, 4, longs, 0, 6, MASK8_2); + private static void decode2(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(4, longs, 6, MASK8_2, tmp, 0, MASK8_6); shiftLongs(tmp, 4, longs, 4, 4, MASK8_2); shiftLongs(tmp, 4, longs, 8, 2, MASK8_2); - shiftLongs(tmp, 4, longs, 12, 0, MASK8_2); + maskLongs(tmp, 4, longs, 12, MASK8_2); } - private static void decode3(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 6); - shiftLongs(tmp, 6, longs, 0, 5, MASK8_3); + private static void decode3(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(6, longs, 5, MASK8_3, tmp, 0, MASK8_5); shiftLongs(tmp, 6, longs, 6, 2, MASK8_3); + maskLongs(tmp, 6, tmp, 0, MASK8_2); for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 2; ++iter, tmpIdx += 3, longsIdx += 2) { - long l0 = (tmp[tmpIdx + 0] & MASK8_2) << 1; + long l0 = tmp[tmpIdx + 0] << 1; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 2; - l1 |= (tmp[tmpIdx + 2] & MASK8_2) << 0; + l1 |= tmp[tmpIdx + 2] << 0; longs[longsIdx + 1] = l1; } } - private static void decode4(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 8); - shiftLongs(tmp, 8, longs, 0, 4, MASK8_4); - shiftLongs(tmp, 8, longs, 8, 0, MASK8_4); + private static void decode4(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(8, longs, 4, MASK8_4, longs, 8, MASK8_4); } - private static void decode5(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 10); - shiftLongs(tmp, 10, longs, 0, 3, MASK8_5); + private static void decode5(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(10, longs, 3, MASK8_5, tmp, 0, MASK8_3); for (int iter = 0, tmpIdx = 0, longsIdx = 10; iter < 2; ++iter, tmpIdx += 5, longsIdx += 3) { - long l0 = (tmp[tmpIdx + 0] & MASK8_3) << 2; + long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 4; - l1 |= (tmp[tmpIdx + 2] & MASK8_3) << 1; + l1 |= tmp[tmpIdx + 2] << 1; l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK8_1; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 3] & MASK8_2) << 3; - l2 |= (tmp[tmpIdx + 4] & MASK8_3) << 0; + l2 |= tmp[tmpIdx + 4] << 0; longs[longsIdx + 2] = l2; } } - private static void decode6(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 12); - shiftLongs(tmp, 12, longs, 0, 2, MASK8_6); - shiftLongs(tmp, 12, tmp, 0, 0, MASK8_2); + private static void decode6(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(12, longs, 2, MASK8_6, tmp, 0, MASK8_2); for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 4; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= tmp[tmpIdx + 1] << 2; @@ -687,10 +697,9 @@ final class ForUtil { } } - private static void decode7(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 14); - shiftLongs(tmp, 14, longs, 0, 1, MASK8_7); - shiftLongs(tmp, 14, tmp, 0, 0, MASK8_1); + private static void decode7(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(14, longs, 1, MASK8_7, tmp, 0, MASK8_1); for (int iter = 0, tmpIdx = 0, longsIdx = 14; iter < 2; ++iter, tmpIdx += 7, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 6; l0 |= tmp[tmpIdx + 1] << 5; @@ -703,15 +712,16 @@ final class ForUtil { } } - private static void decode8(DataInput in, long[] tmp, long[] longs) throws IOException { + private static void decode8(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { in.readLongs(longs, 0, 16); } - private static void decode9(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 18); - shiftLongs(tmp, 18, longs, 0, 7, MASK16_9); + private static void decode9(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(18, longs, 7, MASK16_9, tmp, 0, MASK16_7); for (int iter = 0, tmpIdx = 0, longsIdx = 18; iter < 2; ++iter, tmpIdx += 9, longsIdx += 7) { - long l0 = (tmp[tmpIdx + 0] & MASK16_7) << 2; + long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK16_5) << 4; @@ -721,7 +731,7 @@ final class ForUtil { l2 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_6; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 3] & MASK16_1) << 8; - l3 |= (tmp[tmpIdx + 4] & MASK16_7) << 1; + l3 |= tmp[tmpIdx + 4] << 1; l3 |= (tmp[tmpIdx + 5] >>> 6) & MASK16_1; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 5] & MASK16_6) << 3; @@ -731,59 +741,58 @@ final class ForUtil { l5 |= (tmp[tmpIdx + 7] >>> 2) & MASK16_5; longs[longsIdx + 5] = l5; long l6 = (tmp[tmpIdx + 7] & MASK16_2) << 7; - l6 |= (tmp[tmpIdx + 8] & MASK16_7) << 0; + l6 |= tmp[tmpIdx + 8] << 0; longs[longsIdx + 6] = l6; } } - private static void decode10(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 20); - shiftLongs(tmp, 20, longs, 0, 6, MASK16_10); + private static void decode10(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(20, longs, 6, MASK16_10, tmp, 0, MASK16_6); for (int iter = 0, tmpIdx = 0, longsIdx = 20; iter < 4; ++iter, tmpIdx += 5, longsIdx += 3) { - long l0 = (tmp[tmpIdx + 0] & MASK16_6) << 4; + long l0 = tmp[tmpIdx + 0] << 4; l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK16_2) << 8; - l1 |= (tmp[tmpIdx + 2] & MASK16_6) << 2; + l1 |= tmp[tmpIdx + 2] << 2; l1 |= (tmp[tmpIdx + 3] >>> 4) & MASK16_2; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 3] & MASK16_4) << 6; - l2 |= (tmp[tmpIdx + 4] & MASK16_6) << 0; + l2 |= tmp[tmpIdx + 4] << 0; longs[longsIdx + 2] = l2; } } - private static void decode11(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 22); - shiftLongs(tmp, 22, longs, 0, 5, MASK16_11); + private static void decode11(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(22, longs, 5, MASK16_11, tmp, 0, MASK16_5); for (int iter = 0, tmpIdx = 0, longsIdx = 22; iter < 2; ++iter, tmpIdx += 11, longsIdx += 5) { - long l0 = (tmp[tmpIdx + 0] & MASK16_5) << 6; - l0 |= (tmp[tmpIdx + 1] & MASK16_5) << 1; + long l0 = tmp[tmpIdx + 0] << 6; + l0 |= tmp[tmpIdx + 1] << 1; l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK16_1; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 2] & MASK16_4) << 7; - l1 |= (tmp[tmpIdx + 3] & MASK16_5) << 2; + l1 |= tmp[tmpIdx + 3] << 2; l1 |= (tmp[tmpIdx + 4] >>> 3) & MASK16_2; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 4] & MASK16_3) << 8; - l2 |= (tmp[tmpIdx + 5] & MASK16_5) << 3; + l2 |= tmp[tmpIdx + 5] << 3; l2 |= (tmp[tmpIdx + 6] >>> 2) & MASK16_3; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 6] & MASK16_2) << 9; - l3 |= (tmp[tmpIdx + 7] & MASK16_5) << 4; + l3 |= tmp[tmpIdx + 7] << 4; l3 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_4; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 8] & MASK16_1) << 10; - l4 |= (tmp[tmpIdx + 9] & MASK16_5) << 5; - l4 |= (tmp[tmpIdx + 10] & MASK16_5) << 0; + l4 |= tmp[tmpIdx + 9] << 5; + l4 |= tmp[tmpIdx + 10] << 0; longs[longsIdx + 4] = l4; } } - private static void decode12(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 24); - shiftLongs(tmp, 24, longs, 0, 4, MASK16_12); - shiftLongs(tmp, 24, tmp, 0, 0, MASK16_4); + private static void decode12(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(24, longs, 4, MASK16_12, tmp, 0, MASK16_4); for (int iter = 0, tmpIdx = 0, longsIdx = 24; iter < 8; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 8; l0 |= tmp[tmpIdx + 1] << 4; @@ -792,35 +801,34 @@ final class ForUtil { } } - private static void decode13(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 26); - shiftLongs(tmp, 26, longs, 0, 3, MASK16_13); + private static void decode13(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(26, longs, 3, MASK16_13, tmp, 0, MASK16_3); for (int iter = 0, tmpIdx = 0, longsIdx = 26; iter < 2; ++iter, tmpIdx += 13, longsIdx += 3) { - long l0 = (tmp[tmpIdx + 0] & MASK16_3) << 10; - l0 |= (tmp[tmpIdx + 1] & MASK16_3) << 7; - l0 |= (tmp[tmpIdx + 2] & MASK16_3) << 4; - l0 |= (tmp[tmpIdx + 3] & MASK16_3) << 1; + long l0 = tmp[tmpIdx + 0] << 10; + l0 |= tmp[tmpIdx + 1] << 7; + l0 |= tmp[tmpIdx + 2] << 4; + l0 |= tmp[tmpIdx + 3] << 1; l0 |= (tmp[tmpIdx + 4] >>> 2) & MASK16_1; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 4] & MASK16_2) << 11; - l1 |= (tmp[tmpIdx + 5] & MASK16_3) << 8; - l1 |= (tmp[tmpIdx + 6] & MASK16_3) << 5; - l1 |= (tmp[tmpIdx + 7] & MASK16_3) << 2; + l1 |= tmp[tmpIdx + 5] << 8; + l1 |= tmp[tmpIdx + 6] << 5; + l1 |= tmp[tmpIdx + 7] << 2; l1 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_2; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 8] & MASK16_1) << 12; - l2 |= (tmp[tmpIdx + 9] & MASK16_3) << 9; - l2 |= (tmp[tmpIdx + 10] & MASK16_3) << 6; - l2 |= (tmp[tmpIdx + 11] & MASK16_3) << 3; - l2 |= (tmp[tmpIdx + 12] & MASK16_3) << 0; + l2 |= tmp[tmpIdx + 9] << 9; + l2 |= tmp[tmpIdx + 10] << 6; + l2 |= tmp[tmpIdx + 11] << 3; + l2 |= tmp[tmpIdx + 12] << 0; longs[longsIdx + 2] = l2; } } - private static void decode14(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 28); - shiftLongs(tmp, 28, longs, 0, 2, MASK16_14); - shiftLongs(tmp, 28, tmp, 0, 0, MASK16_2); + private static void decode14(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(28, longs, 2, MASK16_14, tmp, 0, MASK16_2); for (int iter = 0, tmpIdx = 0, longsIdx = 28; iter < 4; ++iter, tmpIdx += 7, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 12; l0 |= tmp[tmpIdx + 1] << 10; @@ -833,10 +841,9 @@ final class ForUtil { } } - private static void decode15(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 30); - shiftLongs(tmp, 30, longs, 0, 1, MASK16_15); - shiftLongs(tmp, 30, tmp, 0, 0, MASK16_1); + private static void decode15(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(30, longs, 1, MASK16_15, tmp, 0, MASK16_1); for (int iter = 0, tmpIdx = 0, longsIdx = 30; iter < 2; ++iter, tmpIdx += 15, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 14; l0 |= tmp[tmpIdx + 1] << 13; @@ -857,15 +864,16 @@ final class ForUtil { } } - private static void decode16(DataInput in, long[] tmp, long[] longs) throws IOException { + private static void decode16(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { in.readLongs(longs, 0, 32); } - private static void decode17(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 34); - shiftLongs(tmp, 34, longs, 0, 15, MASK32_17); + private static void decode17(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(34, longs, 15, MASK32_17, tmp, 0, MASK32_15); for (int iter = 0, tmpIdx = 0, longsIdx = 34; iter < 2; ++iter, tmpIdx += 17, longsIdx += 15) { - long l0 = (tmp[tmpIdx + 0] & MASK32_15) << 2; + long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK32_13) << 4; @@ -887,7 +895,7 @@ final class ForUtil { l6 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_14; longs[longsIdx + 6] = l6; long l7 = (tmp[tmpIdx + 7] & MASK32_1) << 16; - l7 |= (tmp[tmpIdx + 8] & MASK32_15) << 1; + l7 |= tmp[tmpIdx + 8] << 1; l7 |= (tmp[tmpIdx + 9] >>> 14) & MASK32_1; longs[longsIdx + 7] = l7; long l8 = (tmp[tmpIdx + 9] & MASK32_14) << 3; @@ -909,16 +917,16 @@ final class ForUtil { l13 |= (tmp[tmpIdx + 15] >>> 2) & MASK32_13; longs[longsIdx + 13] = l13; long l14 = (tmp[tmpIdx + 15] & MASK32_2) << 15; - l14 |= (tmp[tmpIdx + 16] & MASK32_15) << 0; + l14 |= tmp[tmpIdx + 16] << 0; longs[longsIdx + 14] = l14; } } - private static void decode18(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 36); - shiftLongs(tmp, 36, longs, 0, 14, MASK32_18); + private static void decode18(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(36, longs, 14, MASK32_18, tmp, 0, MASK32_14); for (int iter = 0, tmpIdx = 0, longsIdx = 36; iter < 4; ++iter, tmpIdx += 9, longsIdx += 7) { - long l0 = (tmp[tmpIdx + 0] & MASK32_14) << 4; + long l0 = tmp[tmpIdx + 0] << 4; l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK32_10) << 8; @@ -928,7 +936,7 @@ final class ForUtil { l2 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_12; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 3] & MASK32_2) << 16; - l3 |= (tmp[tmpIdx + 4] & MASK32_14) << 2; + l3 |= tmp[tmpIdx + 4] << 2; l3 |= (tmp[tmpIdx + 5] >>> 12) & MASK32_2; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 5] & MASK32_12) << 6; @@ -938,206 +946,205 @@ final class ForUtil { l5 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_10; longs[longsIdx + 5] = l5; long l6 = (tmp[tmpIdx + 7] & MASK32_4) << 14; - l6 |= (tmp[tmpIdx + 8] & MASK32_14) << 0; + l6 |= tmp[tmpIdx + 8] << 0; longs[longsIdx + 6] = l6; } } - private static void decode19(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 38); - shiftLongs(tmp, 38, longs, 0, 13, MASK32_19); + private static void decode19(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(38, longs, 13, MASK32_19, tmp, 0, MASK32_13); for (int iter = 0, tmpIdx = 0, longsIdx = 38; iter < 2; ++iter, tmpIdx += 19, longsIdx += 13) { - long l0 = (tmp[tmpIdx + 0] & MASK32_13) << 6; + long l0 = tmp[tmpIdx + 0] << 6; l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK32_7) << 12; l1 |= (tmp[tmpIdx + 2] >>> 1) & MASK32_12; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 2] & MASK32_1) << 18; - l2 |= (tmp[tmpIdx + 3] & MASK32_13) << 5; + l2 |= tmp[tmpIdx + 3] << 5; l2 |= (tmp[tmpIdx + 4] >>> 8) & MASK32_5; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 4] & MASK32_8) << 11; l3 |= (tmp[tmpIdx + 5] >>> 2) & MASK32_11; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 5] & MASK32_2) << 17; - l4 |= (tmp[tmpIdx + 6] & MASK32_13) << 4; + l4 |= tmp[tmpIdx + 6] << 4; l4 |= (tmp[tmpIdx + 7] >>> 9) & MASK32_4; longs[longsIdx + 4] = l4; long l5 = (tmp[tmpIdx + 7] & MASK32_9) << 10; l5 |= (tmp[tmpIdx + 8] >>> 3) & MASK32_10; longs[longsIdx + 5] = l5; long l6 = (tmp[tmpIdx + 8] & MASK32_3) << 16; - l6 |= (tmp[tmpIdx + 9] & MASK32_13) << 3; + l6 |= tmp[tmpIdx + 9] << 3; l6 |= (tmp[tmpIdx + 10] >>> 10) & MASK32_3; longs[longsIdx + 6] = l6; long l7 = (tmp[tmpIdx + 10] & MASK32_10) << 9; l7 |= (tmp[tmpIdx + 11] >>> 4) & MASK32_9; longs[longsIdx + 7] = l7; long l8 = (tmp[tmpIdx + 11] & MASK32_4) << 15; - l8 |= (tmp[tmpIdx + 12] & MASK32_13) << 2; + l8 |= tmp[tmpIdx + 12] << 2; l8 |= (tmp[tmpIdx + 13] >>> 11) & MASK32_2; longs[longsIdx + 8] = l8; long l9 = (tmp[tmpIdx + 13] & MASK32_11) << 8; l9 |= (tmp[tmpIdx + 14] >>> 5) & MASK32_8; longs[longsIdx + 9] = l9; long l10 = (tmp[tmpIdx + 14] & MASK32_5) << 14; - l10 |= (tmp[tmpIdx + 15] & MASK32_13) << 1; + l10 |= tmp[tmpIdx + 15] << 1; l10 |= (tmp[tmpIdx + 16] >>> 12) & MASK32_1; longs[longsIdx + 10] = l10; long l11 = (tmp[tmpIdx + 16] & MASK32_12) << 7; l11 |= (tmp[tmpIdx + 17] >>> 6) & MASK32_7; longs[longsIdx + 11] = l11; long l12 = (tmp[tmpIdx + 17] & MASK32_6) << 13; - l12 |= (tmp[tmpIdx + 18] & MASK32_13) << 0; + l12 |= tmp[tmpIdx + 18] << 0; longs[longsIdx + 12] = l12; } } - private static void decode20(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 40); - shiftLongs(tmp, 40, longs, 0, 12, MASK32_20); + private static void decode20(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(40, longs, 12, MASK32_20, tmp, 0, MASK32_12); for (int iter = 0, tmpIdx = 0, longsIdx = 40; iter < 8; ++iter, tmpIdx += 5, longsIdx += 3) { - long l0 = (tmp[tmpIdx + 0] & MASK32_12) << 8; + long l0 = tmp[tmpIdx + 0] << 8; l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK32_4) << 16; - l1 |= (tmp[tmpIdx + 2] & MASK32_12) << 4; + l1 |= tmp[tmpIdx + 2] << 4; l1 |= (tmp[tmpIdx + 3] >>> 8) & MASK32_4; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 3] & MASK32_8) << 12; - l2 |= (tmp[tmpIdx + 4] & MASK32_12) << 0; + l2 |= tmp[tmpIdx + 4] << 0; longs[longsIdx + 2] = l2; } } - private static void decode21(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 42); - shiftLongs(tmp, 42, longs, 0, 11, MASK32_21); + private static void decode21(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(42, longs, 11, MASK32_21, tmp, 0, MASK32_11); for (int iter = 0, tmpIdx = 0, longsIdx = 42; iter < 2; ++iter, tmpIdx += 21, longsIdx += 11) { - long l0 = (tmp[tmpIdx + 0] & MASK32_11) << 10; + long l0 = tmp[tmpIdx + 0] << 10; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 1] & MASK32_1) << 20; - l1 |= (tmp[tmpIdx + 2] & MASK32_11) << 9; + l1 |= tmp[tmpIdx + 2] << 9; l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_9; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 3] & MASK32_2) << 19; - l2 |= (tmp[tmpIdx + 4] & MASK32_11) << 8; + l2 |= tmp[tmpIdx + 4] << 8; l2 |= (tmp[tmpIdx + 5] >>> 3) & MASK32_8; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 5] & MASK32_3) << 18; - l3 |= (tmp[tmpIdx + 6] & MASK32_11) << 7; + l3 |= tmp[tmpIdx + 6] << 7; l3 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_7; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 7] & MASK32_4) << 17; - l4 |= (tmp[tmpIdx + 8] & MASK32_11) << 6; + l4 |= tmp[tmpIdx + 8] << 6; l4 |= (tmp[tmpIdx + 9] >>> 5) & MASK32_6; longs[longsIdx + 4] = l4; long l5 = (tmp[tmpIdx + 9] & MASK32_5) << 16; - l5 |= (tmp[tmpIdx + 10] & MASK32_11) << 5; + l5 |= tmp[tmpIdx + 10] << 5; l5 |= (tmp[tmpIdx + 11] >>> 6) & MASK32_5; longs[longsIdx + 5] = l5; long l6 = (tmp[tmpIdx + 11] & MASK32_6) << 15; - l6 |= (tmp[tmpIdx + 12] & MASK32_11) << 4; + l6 |= tmp[tmpIdx + 12] << 4; l6 |= (tmp[tmpIdx + 13] >>> 7) & MASK32_4; longs[longsIdx + 6] = l6; long l7 = (tmp[tmpIdx + 13] & MASK32_7) << 14; - l7 |= (tmp[tmpIdx + 14] & MASK32_11) << 3; + l7 |= tmp[tmpIdx + 14] << 3; l7 |= (tmp[tmpIdx + 15] >>> 8) & MASK32_3; longs[longsIdx + 7] = l7; long l8 = (tmp[tmpIdx + 15] & MASK32_8) << 13; - l8 |= (tmp[tmpIdx + 16] & MASK32_11) << 2; + l8 |= tmp[tmpIdx + 16] << 2; l8 |= (tmp[tmpIdx + 17] >>> 9) & MASK32_2; longs[longsIdx + 8] = l8; long l9 = (tmp[tmpIdx + 17] & MASK32_9) << 12; - l9 |= (tmp[tmpIdx + 18] & MASK32_11) << 1; + l9 |= tmp[tmpIdx + 18] << 1; l9 |= (tmp[tmpIdx + 19] >>> 10) & MASK32_1; longs[longsIdx + 9] = l9; long l10 = (tmp[tmpIdx + 19] & MASK32_10) << 11; - l10 |= (tmp[tmpIdx + 20] & MASK32_11) << 0; + l10 |= tmp[tmpIdx + 20] << 0; longs[longsIdx + 10] = l10; } } - private static void decode22(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 44); - shiftLongs(tmp, 44, longs, 0, 10, MASK32_22); + private static void decode22(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(44, longs, 10, MASK32_22, tmp, 0, MASK32_10); for (int iter = 0, tmpIdx = 0, longsIdx = 44; iter < 4; ++iter, tmpIdx += 11, longsIdx += 5) { - long l0 = (tmp[tmpIdx + 0] & MASK32_10) << 12; - l0 |= (tmp[tmpIdx + 1] & MASK32_10) << 2; + long l0 = tmp[tmpIdx + 0] << 12; + l0 |= tmp[tmpIdx + 1] << 2; l0 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 2] & MASK32_8) << 14; - l1 |= (tmp[tmpIdx + 3] & MASK32_10) << 4; + l1 |= tmp[tmpIdx + 3] << 4; l1 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 4] & MASK32_6) << 16; - l2 |= (tmp[tmpIdx + 5] & MASK32_10) << 6; + l2 |= tmp[tmpIdx + 5] << 6; l2 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 6] & MASK32_4) << 18; - l3 |= (tmp[tmpIdx + 7] & MASK32_10) << 8; + l3 |= tmp[tmpIdx + 7] << 8; l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 8] & MASK32_2) << 20; - l4 |= (tmp[tmpIdx + 9] & MASK32_10) << 10; - l4 |= (tmp[tmpIdx + 10] & MASK32_10) << 0; + l4 |= tmp[tmpIdx + 9] << 10; + l4 |= tmp[tmpIdx + 10] << 0; longs[longsIdx + 4] = l4; } } - private static void decode23(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 46); - shiftLongs(tmp, 46, longs, 0, 9, MASK32_23); + private static void decode23(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(46, longs, 9, MASK32_23, tmp, 0, MASK32_9); for (int iter = 0, tmpIdx = 0, longsIdx = 46; iter < 2; ++iter, tmpIdx += 23, longsIdx += 9) { - long l0 = (tmp[tmpIdx + 0] & MASK32_9) << 14; - l0 |= (tmp[tmpIdx + 1] & MASK32_9) << 5; + long l0 = tmp[tmpIdx + 0] << 14; + l0 |= tmp[tmpIdx + 1] << 5; l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK32_5; longs[longsIdx + 0] = l0; long l1 = (tmp[tmpIdx + 2] & MASK32_4) << 19; - l1 |= (tmp[tmpIdx + 3] & MASK32_9) << 10; - l1 |= (tmp[tmpIdx + 4] & MASK32_9) << 1; + l1 |= tmp[tmpIdx + 3] << 10; + l1 |= tmp[tmpIdx + 4] << 1; l1 |= (tmp[tmpIdx + 5] >>> 8) & MASK32_1; longs[longsIdx + 1] = l1; long l2 = (tmp[tmpIdx + 5] & MASK32_8) << 15; - l2 |= (tmp[tmpIdx + 6] & MASK32_9) << 6; + l2 |= tmp[tmpIdx + 6] << 6; l2 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_6; longs[longsIdx + 2] = l2; long l3 = (tmp[tmpIdx + 7] & MASK32_3) << 20; - l3 |= (tmp[tmpIdx + 8] & MASK32_9) << 11; - l3 |= (tmp[tmpIdx + 9] & MASK32_9) << 2; + l3 |= tmp[tmpIdx + 8] << 11; + l3 |= tmp[tmpIdx + 9] << 2; l3 |= (tmp[tmpIdx + 10] >>> 7) & MASK32_2; longs[longsIdx + 3] = l3; long l4 = (tmp[tmpIdx + 10] & MASK32_7) << 16; - l4 |= (tmp[tmpIdx + 11] & MASK32_9) << 7; + l4 |= tmp[tmpIdx + 11] << 7; l4 |= (tmp[tmpIdx + 12] >>> 2) & MASK32_7; longs[longsIdx + 4] = l4; long l5 = (tmp[tmpIdx + 12] & MASK32_2) << 21; - l5 |= (tmp[tmpIdx + 13] & MASK32_9) << 12; - l5 |= (tmp[tmpIdx + 14] & MASK32_9) << 3; + l5 |= tmp[tmpIdx + 13] << 12; + l5 |= tmp[tmpIdx + 14] << 3; l5 |= (tmp[tmpIdx + 15] >>> 6) & MASK32_3; longs[longsIdx + 5] = l5; long l6 = (tmp[tmpIdx + 15] & MASK32_6) << 17; - l6 |= (tmp[tmpIdx + 16] & MASK32_9) << 8; + l6 |= tmp[tmpIdx + 16] << 8; l6 |= (tmp[tmpIdx + 17] >>> 1) & MASK32_8; longs[longsIdx + 6] = l6; long l7 = (tmp[tmpIdx + 17] & MASK32_1) << 22; - l7 |= (tmp[tmpIdx + 18] & MASK32_9) << 13; - l7 |= (tmp[tmpIdx + 19] & MASK32_9) << 4; + l7 |= tmp[tmpIdx + 18] << 13; + l7 |= tmp[tmpIdx + 19] << 4; l7 |= (tmp[tmpIdx + 20] >>> 5) & MASK32_4; longs[longsIdx + 7] = l7; long l8 = (tmp[tmpIdx + 20] & MASK32_5) << 18; - l8 |= (tmp[tmpIdx + 21] & MASK32_9) << 9; - l8 |= (tmp[tmpIdx + 22] & MASK32_9) << 0; + l8 |= tmp[tmpIdx + 21] << 9; + l8 |= tmp[tmpIdx + 22] << 0; longs[longsIdx + 8] = l8; } } - private static void decode24(DataInput in, long[] tmp, long[] longs) throws IOException { - in.readLongs(tmp, 0, 48); - shiftLongs(tmp, 48, longs, 0, 8, MASK32_24); - shiftLongs(tmp, 48, tmp, 0, 0, MASK32_8); + private static void decode24(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) + throws IOException { + pdu.splitLongs(48, longs, 8, MASK32_24, tmp, 0, MASK32_8); for (int iter = 0, tmpIdx = 0, longsIdx = 48; iter < 16; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 16; l0 |= tmp[tmpIdx + 1] << 8; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index 5e66a200929..491b4507cf1 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -352,6 +352,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { final IndexInput startDocIn; IndexInput docIn; + PostingIndexInput postingDocIn; final boolean indexHasFreq; final boolean indexHasPos; final boolean indexHasOffsetsOrPayloads; @@ -413,6 +414,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (docIn == null) { // lazy init docIn = startDocIn.clone(); + postingDocIn = new PostingIndexInput(docIn, forUtil); } prefetchPostings(docIn, termState); } @@ -446,7 +448,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public int freq() throws IOException { if (freqFP != -1) { docIn.seek(freqFP); - pforUtil.decode(docIn, freqBuffer); + pforUtil.decode(postingDocIn, freqBuffer); freqFP = -1; } @@ -481,7 +483,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private void refillFullBlock() throws IOException { assert docFreq - docCountUpto >= BLOCK_SIZE; - forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + forDeltaUtil.decodeAndPrefixSum(postingDocIn, prevDocID, docBuffer); if (indexHasFreq) { if (needsFreq) { @@ -649,8 +651,11 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { final IndexInput startDocIn; IndexInput docIn; + PostingIndexInput postingDocIn; final IndexInput posIn; + final PostingIndexInput postingPosIn; final IndexInput payIn; + final PostingIndexInput postingPayIn; final BytesRef payload; final boolean indexHasFreq; @@ -718,10 +723,13 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; this.posIn = Lucene912PostingsReader.this.posIn.clone(); + postingPosIn = new PostingIndexInput(posIn, forUtil); if (indexHasOffsetsOrPayloads) { this.payIn = Lucene912PostingsReader.this.payIn.clone(); + postingPayIn = new PostingIndexInput(payIn, forUtil); } else { this.payIn = null; + postingPayIn = null; } if (indexHasOffsets) { offsetStartDeltaBuffer = new long[BLOCK_SIZE]; @@ -768,6 +776,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (docIn == null) { // lazy init docIn = startDocIn.clone(); + postingDocIn = new PostingIndexInput(docIn, forUtil); } prefetchPostings(docIn, termState); } @@ -830,8 +839,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); - pforUtil.decode(docIn, freqBuffer); + forDeltaUtil.decodeAndPrefixSum(postingDocIn, prevDocID, docBuffer); + pforUtil.decode(postingDocIn, freqBuffer); docCountUpto += BLOCK_SIZE; } else if (docFreq == 1) { docBuffer[0] = singletonDocID; @@ -1110,11 +1119,11 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } payloadByteUpto = 0; } else { - pforUtil.decode(posIn, posDeltaBuffer); + pforUtil.decode(postingPosIn, posDeltaBuffer); if (indexHasPayloads) { if (needsPayloads) { - pforUtil.decode(payIn, payloadLengthBuffer); + pforUtil.decode(postingPayIn, payloadLengthBuffer); int numBytes = payIn.readVInt(); if (numBytes > payloadBytes.length) { @@ -1133,8 +1142,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (indexHasOffsets) { if (needsOffsets) { - pforUtil.decode(payIn, offsetStartDeltaBuffer); - pforUtil.decode(payIn, offsetLengthBuffer); + pforUtil.decode(postingPayIn, offsetStartDeltaBuffer); + pforUtil.decode(postingPayIn, offsetLengthBuffer); } else { // this works, because when writing a vint block we always force the first length to be // written @@ -1217,7 +1226,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { final IndexInput startDocIn; - IndexInput docIn; + final IndexInput docIn; + final PostingIndexInput postingDocIn; final boolean indexHasFreq; final boolean indexHasPos; final boolean indexHasOffsetsOrPayloads; @@ -1248,7 +1258,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public BlockImpactsDocsEnum(FieldInfo fieldInfo, IntBlockTermState termState) throws IOException { this.startDocIn = Lucene912PostingsReader.this.docIn; - this.docIn = null; indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; @@ -1264,11 +1273,12 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docFreq = termState.docFreq; if (docFreq > 1) { - if (docIn == null) { - // lazy init - docIn = startDocIn.clone(); - } + docIn = startDocIn.clone(); + postingDocIn = new PostingIndexInput(docIn, forUtil); prefetchPostings(docIn, termState); + } else { + docIn = null; + postingDocIn = null; } doc = -1; @@ -1302,7 +1312,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public int freq() throws IOException { if (freqFP != -1) { docIn.seek(freqFP); - pforUtil.decode(docIn, freqBuffer); + pforUtil.decode(postingDocIn, freqBuffer); freqFP = -1; } return (int) freqBuffer[docBufferUpto - 1]; @@ -1338,7 +1348,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + forDeltaUtil.decodeAndPrefixSum(postingDocIn, prevDocID, docBuffer); if (indexHasFreq) { freqFP = docIn.getFilePointer(); @@ -1573,8 +1583,10 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { final IndexInput startDocIn; - IndexInput docIn; + final IndexInput docIn; + final PostingIndexInput postingDocIn; final IndexInput posIn; + final PostingIndexInput postingPosIn; final boolean indexHasFreq; final boolean indexHasPos; @@ -1628,7 +1640,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState) throws IOException { this.startDocIn = Lucene912PostingsReader.this.docIn; - this.docIn = null; indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; @@ -1641,6 +1652,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; this.posIn = Lucene912PostingsReader.this.posIn.clone(); + postingPosIn = new PostingIndexInput(posIn, forUtil); // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in // advance() @@ -1651,11 +1663,12 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { totalTermFreq = termState.totalTermFreq; singletonDocID = termState.singletonDocID; if (docFreq > 1) { - if (docIn == null) { - // lazy init - docIn = startDocIn.clone(); - } + docIn = startDocIn.clone(); + postingDocIn = new PostingIndexInput(docIn, forUtil); prefetchPostings(docIn, termState); + } else { + docIn = null; + postingDocIn = null; } posIn.seek(posTermStartFP); level1PosEndFP = posTermStartFP; @@ -1707,8 +1720,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); - pforUtil.decode(docIn, freqBuffer); + forDeltaUtil.decodeAndPrefixSum(postingDocIn, prevDocID, docBuffer); + pforUtil.decode(postingDocIn, freqBuffer); docCountUpto += BLOCK_SIZE; } else if (docFreq == 1) { docBuffer[0] = singletonDocID; @@ -1981,7 +1994,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } } else { - pforUtil.decode(posIn, posDeltaBuffer); + pforUtil.decode(postingPosIn, posDeltaBuffer); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java index b307080b215..b3c6503449a 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java @@ -16,10 +16,12 @@ */ package org.apache.lucene.codecs.lucene912; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.*; import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_MASK; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC; import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java index f4405ae66fa..4fbe7051f3b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java @@ -104,18 +104,18 @@ final class PForUtil { } /** Decode 128 integers into {@code ints}. */ - void decode(DataInput in, long[] longs) throws IOException { - final int token = Byte.toUnsignedInt(in.readByte()); + void decode(PostingIndexInput in, long[] longs) throws IOException { + final int token = Byte.toUnsignedInt(in.in.readByte()); final int bitsPerValue = token & 0x1f; final int numExceptions = token >>> 5; if (bitsPerValue == 0) { - Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong()); + Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.in.readVLong()); } else { - forUtil.decode(bitsPerValue, in, longs); + in.decode(bitsPerValue, longs); } for (int i = 0; i < numExceptions; ++i) { - longs[Byte.toUnsignedInt(in.readByte())] |= - Byte.toUnsignedLong(in.readByte()) << bitsPerValue; + longs[Byte.toUnsignedInt(in.in.readByte())] |= + Byte.toUnsignedLong(in.in.readByte()) << bitsPerValue; } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java new file mode 100644 index 00000000000..88067b02858 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java @@ -0,0 +1,55 @@ +/* + * 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.codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.internal.vectorization.VectorizationProvider; +import org.apache.lucene.store.IndexInput; + +/** + * Wrapper around an {@link IndexInput} and a {@link ForUtil} that optionally optimizes decoding + * using vectorization. + */ +public final class PostingIndexInput { + + private static final VectorizationProvider VECTORIZATION_PROVIDER = + VectorizationProvider.getInstance(); + + public final IndexInput in; + public final ForUtil forUtil; + private final PostingDecodingUtil postingDecodingUtil; + + public PostingIndexInput(IndexInput in, ForUtil forUtil) throws IOException { + this.in = in; + this.forUtil = forUtil; + this.postingDecodingUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); + } + + /** Decode 128 integers stored on {@code bitsPerValues} bits per value into {@code longs}. */ + public void decode(int bitsPerValue, long[] longs) throws IOException { + forUtil.decode(bitsPerValue, in, postingDecodingUtil, longs); + } + + /** + * Decode 128 integers stored on {@code bitsPerValues} bits per value, compute their prefix sum, + * and store results into {@code longs}. + */ + public void decodeAndPrefixSum(int bitsPerValue, long base, long[] longs) throws IOException { + forUtil.decodeAndPrefixSum(bitsPerValue, in, postingDecodingUtil, base, longs); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py index c6a33ceef53..5e993e26555 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py @@ -43,17 +43,20 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT package org.apache.lucene.codecs.lucene912; import java.io.IOException; -import org.apache.lucene.store.DataInput; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; -// Inspired from https://fulmicoton.com/posts/bitpacking/ -// Encodes multiple integers in a long to get SIMD-like speedups. -// If bitsPerValue <= 8 then we pack 8 ints per long -// else if bitsPerValue <= 16 we pack 4 ints per long -// else we pack 2 ints per long -final class ForUtil { +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ + * Encodes multiple integers in a long to get SIMD-like speedups. + * If bitsPerValue <= 8 then we pack 8 ints per long + * else if bitsPerValue <= 16 we pack 4 ints per long + * else we pack 2 ints per long + */ +public final class ForUtil { - static final int BLOCK_SIZE = 128; + public static final int BLOCK_SIZE = 128; private static final int BLOCK_SIZE_LOG2 = 7; private static long expandMask32(long mask32) { @@ -324,13 +327,13 @@ final class ForUtil { return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); } - private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + private static void decodeSlow(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { final int numLongs = bitsPerValue << 1; - in.readLongs(tmp, 0, numLongs); final long mask = MASKS32[bitsPerValue]; - int longsIdx = 0; - int shift = 32 - bitsPerValue; + pdu.splitLongs(numLongs, longs, 32 - bitsPerValue, mask, tmp, 0, -1L); + int longsIdx = numLongs; + int shift = 32 - 2 * bitsPerValue; for (; shift >= 0; shift -= bitsPerValue) { shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); longsIdx += numLongs; @@ -366,31 +369,17 @@ final class ForUtil { } } + /** + * Likewise, but for a simple mask. + */ + private static void maskLongs(long[] a, int count, long[] b, int bi, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = a[i] & mask; + } + } + """ -def writeRemainderWithSIMDOptimize(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): - iteration = 1 - num_longs = bpv * num_values / remaining_bits_per_long - while num_longs % 2 == 0 and num_values % 2 == 0: - num_longs /= 2 - num_values /= 2 - iteration *= 2 - - f.write(' shiftLongs(tmp, %d, tmp, 0, 0, MASK%d_%d);\n' % (iteration * num_longs, next_primitive, remaining_bits_per_long)) - f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values)) - tmp_idx = 0 - b = bpv - b -= remaining_bits_per_long - f.write(' long l0 = tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) - tmp_idx += 1 - while b >= remaining_bits_per_long: - b -= remaining_bits_per_long - f.write(' l0 |= tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) - tmp_idx += 1 - f.write(' longs[longsIdx + 0] = l0;\n') - f.write(' }\n') - - def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): iteration = 1 num_longs = bpv * num_values / remaining_bits_per_long @@ -406,14 +395,14 @@ def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, b = bpv if remaining_bits == 0: b -= remaining_bits_per_long - f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b)) + f.write(' long l%d = tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) else: b -= remaining_bits f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b)) tmp_idx += 1 while b >= remaining_bits_per_long: b -= remaining_bits_per_long - f.write(' l%d |= (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b)) + f.write(' l%d |= tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) tmp_idx += 1 if b > 0: f.write(' l%d |= (tmp[tmpIdx + %d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_long-b, next_primitive, b)) @@ -428,23 +417,30 @@ def writeDecode(bpv, f): next_primitive = 8 elif bpv <= 16: next_primitive = 16 - f.write(' private static void decode%d(DataInput in, long[] tmp, long[] longs) throws IOException {\n' %bpv) - num_values_per_long = 64 / next_primitive + f.write(' private static void decode%d(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %bpv) if bpv == next_primitive: f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2)) + elif bpv * 2 == next_primitive: + f.write(' pdu.splitLongs(%d, longs, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, next_primitive, bpv, bpv*2, next_primitive, next_primitive - bpv)) else: - f.write(' in.readLongs(tmp, 0, %d);\n' %(bpv*2)) - shift = next_primitive - bpv - o = 0 + num_values_per_long = 64 / next_primitive + f.write(' pdu.splitLongs(%d, longs, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, next_primitive, bpv, next_primitive, next_primitive - bpv)) + + shift = next_primitive - 2 * bpv + o = 2 * bpv while shift >= 0: - f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv)) + if shift == 0: + f.write(' maskLongs(tmp, %d, longs, %d, MASK%d_%d);\n' %(bpv*2, o, next_primitive, bpv)) + else: + f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv)) o += bpv*2 shift -= bpv - if shift + bpv > 0: - if bpv % (next_primitive % bpv) == 0: - writeRemainderWithSIMDOptimize(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) - else: - writeRemainder(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) + remaining_bits = shift + bpv + if remaining_bits > 0: + if remaining_bits != next_primitive - bpv: + # values in tmp still have more bits per value than remaining_bits, clear the higher bits now + f.write(' maskLongs(tmp, %d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive, remaining_bits)) + writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_long - o, f) f.write(' }\n') @@ -471,7 +467,7 @@ if __name__ == '__main__': f.write(""" /** Decode 128 integers into {@code longs}. */ - void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + void decode(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] longs) throws IOException { switch (bitsPerValue) { """) for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): @@ -481,11 +477,11 @@ if __name__ == '__main__': elif bpv <= 16: next_primitive = 16 f.write(' case %d:\n' %bpv) - f.write(' decode%d(in, tmp, longs);\n' %bpv) + f.write(' decode%d(in, pdu, tmp, longs);\n' %bpv) f.write(' expand%d(longs);\n' %next_primitive) f.write(' break;\n') f.write(' default:\n') - f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') + f.write(' decodeSlow(bitsPerValue, in, pdu, tmp, longs);\n') f.write(' expand32(longs);\n') f.write(' break;\n') f.write(' }\n') @@ -495,7 +491,7 @@ if __name__ == '__main__': /** * Delta-decode 128 integers into {@code longs}. */ - void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) throws IOException { + void decodeAndPrefixSum(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long base, long[] longs) throws IOException { switch (bitsPerValue) { """) for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): @@ -505,11 +501,11 @@ if __name__ == '__main__': elif bpv <= 16: next_primitive = 16 f.write(' case %d:\n' %bpv) - f.write(' decode%d(in, tmp, longs);\n' %bpv) + f.write(' decode%d(in, pdu, tmp, longs);\n' %bpv) f.write(' prefixSum%d(longs, base);\n' %next_primitive) f.write(' break;\n') f.write(' default:\n') - f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') + f.write(' decodeSlow(bitsPerValue, in, pdu, tmp, longs);\n') f.write(' prefixSum32(longs, base);\n') f.write(' break;\n') f.write(' }\n') diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultPostingDecodingUtil.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultPostingDecodingUtil.java new file mode 100644 index 00000000000..8c68e87109c --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultPostingDecodingUtil.java @@ -0,0 +1,42 @@ +/* + * 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.internal.vectorization; + +import java.io.IOException; +import org.apache.lucene.store.IndexInput; + +final class DefaultPostingDecodingUtil extends PostingDecodingUtil { + + protected final IndexInput in; + + public DefaultPostingDecodingUtil(IndexInput in) { + this.in = in; + } + + @Override + public void splitLongs( + int count, long[] b, int bShift, long bMask, long[] c, int cIndex, long cMask) + throws IOException { + assert count <= 64; + in.readLongs(c, cIndex, count); + // The below loop is auto-vectorized + for (int i = 0; i < count; ++i) { + b[i] = (c[cIndex + i] >>> bShift) & bMask; + c[cIndex + i] &= cMask; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorizationProvider.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorizationProvider.java index c5193aa23de..2127a594117 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorizationProvider.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorizationProvider.java @@ -19,6 +19,7 @@ package org.apache.lucene.internal.vectorization; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; +import org.apache.lucene.store.IndexInput; /** Default provider returning scalar implementations. */ final class DefaultVectorizationProvider extends VectorizationProvider { @@ -38,4 +39,9 @@ final class DefaultVectorizationProvider extends VectorizationProvider { public FlatVectorsScorer getLucene99FlatVectorsScorer() { return DefaultFlatVectorScorer.INSTANCE; } + + @Override + public PostingDecodingUtil newPostingDecodingUtil(IndexInput input) { + return new DefaultPostingDecodingUtil(input); + } } diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java new file mode 100644 index 00000000000..d5928959e28 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java @@ -0,0 +1,32 @@ +/* + * 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.internal.vectorization; + +import java.io.IOException; + +/** Utility class to decode postings. */ +public abstract class PostingDecodingUtil { + + /** + * Read {@code count} longs. This number must not exceed 64. Apply shift {@code bShift} and mask + * {@code bMask} and store the result in {@code b} starting at offset 0. Apply mask {@code cMask} + * and store the result in {@code c} starting at offset {@code cIndex}. + */ + public abstract void splitLongs( + int count, long[] b, int bShift, long bMask, long[] c, int cIndex, long cMask) + throws IOException; +} diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java index a236c303eb4..eeb1830fc91 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java @@ -17,6 +17,7 @@ package org.apache.lucene.internal.vectorization; +import java.io.IOException; import java.lang.StackWalker.StackFrame; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; @@ -28,6 +29,7 @@ import java.util.function.Predicate; import java.util.logging.Logger; import java.util.stream.Stream; import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.Constants; import org.apache.lucene.util.VectorUtil; @@ -95,6 +97,9 @@ public abstract class VectorizationProvider { /** Returns a FlatVectorsScorer that supports the Lucene99 format. */ public abstract FlatVectorsScorer getLucene99FlatVectorsScorer(); + /** Create a new {@link PostingDecodingUtil} for the given {@link IndexInput}. */ + public abstract PostingDecodingUtil newPostingDecodingUtil(IndexInput input) throws IOException; + // *** Lookup mechanism: *** private static final Logger LOG = Logger.getLogger(VectorizationProvider.class.getName()); @@ -184,7 +189,8 @@ public abstract class VectorizationProvider { private static final Set VALID_CALLERS = Set.of( "org.apache.lucene.codecs.hnsw.FlatVectorScorerUtil", - "org.apache.lucene.util.VectorUtil"); + "org.apache.lucene.util.VectorUtil", + "org.apache.lucene.codecs.lucene912.PostingIndexInput"); private static void ensureCaller() { final boolean validCaller = diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java new file mode 100644 index 00000000000..7b4bc32bccf --- /dev/null +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java @@ -0,0 +1,85 @@ +/* + * 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.internal.vectorization; + +import java.io.IOException; +import java.lang.foreign.MemorySegment; +import java.nio.ByteOrder; +import jdk.incubator.vector.LongVector; +import jdk.incubator.vector.VectorOperators; +import jdk.incubator.vector.VectorSpecies; +import org.apache.lucene.store.IndexInput; + +final class MemorySegmentPostingDecodingUtil extends PostingDecodingUtil { + + private static final VectorSpecies LONG_SPECIES = + PanamaVectorConstants.PRERERRED_LONG_SPECIES; + + private final IndexInput in; + private final MemorySegment memorySegment; + + MemorySegmentPostingDecodingUtil(IndexInput in, MemorySegment memorySegment) { + this.in = in; + this.memorySegment = memorySegment; + } + + @Override + public void splitLongs( + int count, long[] b, int bShift, long bMask, long[] c, int cIndex, long cMask) + throws IOException { + if (count < LONG_SPECIES.length()) { + // Not enough data to vectorize without going out-of-bounds. In practice, this branch is never + // used if the bit width is 256, and is used for 2 and 3 bits per value if the bit width is + // 512. + in.readLongs(c, cIndex, count); + for (int i = 0; i < count; ++i) { + b[i] = (c[cIndex + i] >>> bShift) & bMask; + c[cIndex + i] &= cMask; + } + } else { + long offset = in.getFilePointer(); + long endOffset = offset + count * Long.BYTES; + int loopBound = LONG_SPECIES.loopBound(count - 1); + for (int i = 0; + i < loopBound; + i += LONG_SPECIES.length(), offset += LONG_SPECIES.length() * Long.BYTES) { + LongVector vector = + LongVector.fromMemorySegment( + LONG_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); + vector + .lanewise(VectorOperators.LSHR, bShift) + .lanewise(VectorOperators.AND, bMask) + .intoArray(b, i); + vector.lanewise(VectorOperators.AND, cMask).intoArray(c, cIndex + i); + } + + // Handle the tail by reading a vector that is aligned with with `count` on the right side. + int i = count - LONG_SPECIES.length(); + offset = endOffset - LONG_SPECIES.length() * Long.BYTES; + LongVector vector = + LongVector.fromMemorySegment( + LONG_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); + vector + .lanewise(VectorOperators.LSHR, bShift) + .lanewise(VectorOperators.AND, bMask) + .intoArray(b, i); + vector.lanewise(VectorOperators.AND, cMask).intoArray(c, cIndex + i); + + in.seek(endOffset); + } + } +} diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorConstants.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorConstants.java new file mode 100644 index 00000000000..e0c5bbca38e --- /dev/null +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorConstants.java @@ -0,0 +1,55 @@ +/* + * 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.internal.vectorization; + +import jdk.incubator.vector.VectorShape; +import jdk.incubator.vector.VectorSpecies; +import org.apache.lucene.util.Constants; + +/** Shared constants for implementations that take advantage of the Panama Vector API. */ +final class PanamaVectorConstants { + + /** Preferred width in bits for vectors. */ + static final int PREFERRED_VECTOR_BITSIZE; + + /** Whether integer vectors can be trusted to actually be fast. */ + static final boolean HAS_FAST_INTEGER_VECTORS; + + static final VectorSpecies PRERERRED_LONG_SPECIES; + static final VectorSpecies PRERERRED_INT_SPECIES; + + static { + // default to platform supported bitsize + int vectorBitSize = VectorShape.preferredShape().vectorBitSize(); + // but allow easy overriding for testing + PREFERRED_VECTOR_BITSIZE = VectorizationProvider.TESTS_VECTOR_SIZE.orElse(vectorBitSize); + + // hotspot misses some SSE intrinsics, workaround it + // to be fair, they do document this thing only works well with AVX2/AVX3 and Neon + boolean isAMD64withoutAVX2 = + Constants.OS_ARCH.equals("amd64") && PREFERRED_VECTOR_BITSIZE < 256; + HAS_FAST_INTEGER_VECTORS = + VectorizationProvider.TESTS_FORCE_INTEGER_VECTORS || (isAMD64withoutAVX2 == false); + + PRERERRED_LONG_SPECIES = + VectorSpecies.of(long.class, VectorShape.forBitSize(PREFERRED_VECTOR_BITSIZE)); + PRERERRED_INT_SPECIES = + VectorSpecies.of(int.class, VectorShape.forBitSize(PREFERRED_VECTOR_BITSIZE)); + } + + private PanamaVectorConstants() {} +} diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java index 867d0c684cb..ad2dff11cea 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java @@ -52,20 +52,15 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { // preferred vector sizes, which can be altered for testing private static final VectorSpecies FLOAT_SPECIES; - private static final VectorSpecies INT_SPECIES; + private static final VectorSpecies INT_SPECIES = + PanamaVectorConstants.PRERERRED_INT_SPECIES; private static final VectorSpecies BYTE_SPECIES; private static final VectorSpecies SHORT_SPECIES; static final int VECTOR_BITSIZE; - static final boolean HAS_FAST_INTEGER_VECTORS; static { - // default to platform supported bitsize - int vectorBitSize = VectorShape.preferredShape().vectorBitSize(); - // but allow easy overriding for testing - vectorBitSize = VectorizationProvider.TESTS_VECTOR_SIZE.orElse(vectorBitSize); - INT_SPECIES = VectorSpecies.of(int.class, VectorShape.forBitSize(vectorBitSize)); - VECTOR_BITSIZE = INT_SPECIES.vectorBitSize(); + VECTOR_BITSIZE = PanamaVectorConstants.PREFERRED_VECTOR_BITSIZE; FLOAT_SPECIES = INT_SPECIES.withLanes(float.class); // compute BYTE/SHORT sizes relative to preferred integer vector size if (VECTOR_BITSIZE >= 256) { @@ -76,11 +71,6 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { BYTE_SPECIES = null; SHORT_SPECIES = null; } - // hotspot misses some SSE intrinsics, workaround it - // to be fair, they do document this thing only works well with AVX2/AVX3 and Neon - boolean isAMD64withoutAVX2 = Constants.OS_ARCH.equals("amd64") && VECTOR_BITSIZE < 256; - HAS_FAST_INTEGER_VECTORS = - VectorizationProvider.TESTS_FORCE_INTEGER_VECTORS || (isAMD64withoutAVX2 == false); } // the way FMA should work! if available use it, otherwise fall back to mul/add @@ -320,7 +310,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { // only vectorize if we'll at least enter the loop a single time, and we have at least 128-bit // vectors (256-bit on intel to dodge performance landmines) - if (a.byteSize() >= 16 && HAS_FAST_INTEGER_VECTORS) { + if (a.byteSize() >= 16 && PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS) { // compute vectorized dot product consistent with VPDPBUSD instruction if (VECTOR_BITSIZE >= 512) { i += BYTE_SPECIES.loopBound(a.byteSize()); @@ -414,7 +404,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { } else if (VECTOR_BITSIZE == 256) { i += ByteVector.SPECIES_128.loopBound(packed.length); res += dotProductBody256Int4Packed(unpacked, packed, i); - } else if (HAS_FAST_INTEGER_VECTORS) { + } else if (PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS) { i += ByteVector.SPECIES_64.loopBound(packed.length); res += dotProductBody128Int4Packed(unpacked, packed, i); } @@ -430,7 +420,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { } else { if (VECTOR_BITSIZE >= 512 || VECTOR_BITSIZE == 256) { return dotProduct(a, b); - } else if (a.length >= 32 && HAS_FAST_INTEGER_VECTORS) { + } else if (a.length >= 32 && PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS) { i += ByteVector.SPECIES_128.loopBound(a.length); res += int4DotProductBody128(a, b, i); } @@ -588,7 +578,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { // only vectorize if we'll at least enter the loop a single time, and we have at least 128-bit // vectors (256-bit on intel to dodge performance landmines) - if (a.byteSize() >= 16 && HAS_FAST_INTEGER_VECTORS) { + if (a.byteSize() >= 16 && PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS) { final float[] ret; if (VECTOR_BITSIZE >= 512) { i += BYTE_SPECIES.loopBound((int) a.byteSize()); @@ -711,7 +701,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { // only vectorize if we'll at least enter the loop a single time, and we have at least 128-bit // vectors (256-bit on intel to dodge performance landmines) - if (a.byteSize() >= 16 && HAS_FAST_INTEGER_VECTORS) { + if (a.byteSize() >= 16 && PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS) { if (VECTOR_BITSIZE >= 256) { i += BYTE_SPECIES.loopBound((int) a.byteSize()); res += squareDistanceBody256(a, b, i); diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorizationProvider.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorizationProvider.java index 87f7cf2baf7..0e060586c2a 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorizationProvider.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorizationProvider.java @@ -16,19 +16,25 @@ */ package org.apache.lucene.internal.vectorization; +import java.io.IOException; +import java.lang.foreign.MemorySegment; import java.security.AccessController; import java.security.PrivilegedAction; import java.util.Locale; import java.util.logging.Logger; import jdk.incubator.vector.FloatVector; import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.MemorySegmentAccessInput; import org.apache.lucene.util.Constants; import org.apache.lucene.util.SuppressForbidden; /** A vectorization provider that leverages the Panama Vector API. */ final class PanamaVectorizationProvider extends VectorizationProvider { - private final VectorUtilSupport vectorUtilSupport; + // NOTE: Avoid static fields or initializers which rely on the vector API, as these initializers + // would get called before we have a chance to perform sanity checks around the vector API in the + // constructor of this class. Put them in PanamaVectorConstants instead. // Extracted to a method to be able to apply the SuppressForbidden annotation @SuppressWarnings("removal") @@ -37,6 +43,8 @@ final class PanamaVectorizationProvider extends VectorizationProvider { return AccessController.doPrivileged(action); } + private final VectorUtilSupport vectorUtilSupport; + PanamaVectorizationProvider() { // hack to work around for JDK-8309727: try { @@ -51,9 +59,9 @@ final class PanamaVectorizationProvider extends VectorizationProvider { "We hit initialization failure described in JDK-8309727: " + se); } - if (PanamaVectorUtilSupport.VECTOR_BITSIZE < 128) { + if (PanamaVectorConstants.PREFERRED_VECTOR_BITSIZE < 128) { throw new UnsupportedOperationException( - "Vector bit size is less than 128: " + PanamaVectorUtilSupport.VECTOR_BITSIZE); + "Vector bit size is less than 128: " + PanamaVectorConstants.PREFERRED_VECTOR_BITSIZE); } this.vectorUtilSupport = new PanamaVectorUtilSupport(); @@ -63,11 +71,9 @@ final class PanamaVectorizationProvider extends VectorizationProvider { String.format( Locale.ENGLISH, "Java vector incubator API enabled; uses preferredBitSize=%d%s%s", - PanamaVectorUtilSupport.VECTOR_BITSIZE, + PanamaVectorConstants.PREFERRED_VECTOR_BITSIZE, Constants.HAS_FAST_VECTOR_FMA ? "; FMA enabled" : "", - PanamaVectorUtilSupport.HAS_FAST_INTEGER_VECTORS - ? "" - : "; floating-point vectors only")); + PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS ? "" : "; floating-point vectors only")); } @Override @@ -79,4 +85,16 @@ final class PanamaVectorizationProvider extends VectorizationProvider { public FlatVectorsScorer getLucene99FlatVectorsScorer() { return Lucene99MemorySegmentFlatVectorsScorer.INSTANCE; } + + @Override + public PostingDecodingUtil newPostingDecodingUtil(IndexInput input) throws IOException { + if (PanamaVectorConstants.HAS_FAST_INTEGER_VECTORS + && input instanceof MemorySegmentAccessInput msai) { + MemorySegment ms = msai.segmentSliceOrNull(0, input.length()); + if (ms != null) { + return new MemorySegmentPostingDecodingUtil(input, ms); + } + } + return new DefaultPostingDecodingUtil(input); + } } diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentAccessInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentAccessInput.java index 7c22eccdcf1..8b6452a748b 100644 --- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentAccessInput.java +++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentAccessInput.java @@ -27,7 +27,7 @@ import java.lang.foreign.MemorySegment; public interface MemorySegmentAccessInput extends RandomAccessInput, Cloneable { /** Returns the memory segment for a given position and length, or null. */ - MemorySegment segmentSliceOrNull(long pos, int len) throws IOException; + MemorySegment segmentSliceOrNull(long pos, long len) throws IOException; MemorySegmentAccessInput clone(); } diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java index e9805f0f7a6..c6ac3d23a12 100644 --- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java +++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java @@ -742,7 +742,7 @@ abstract class MemorySegmentIndexInput extends IndexInput } @Override - public MemorySegment segmentSliceOrNull(long pos, int len) throws IOException { + public MemorySegment segmentSliceOrNull(long pos, long len) throws IOException { try { Objects.checkIndex(pos + len, this.length + 1); return curSegment.asSlice(pos, len); @@ -816,7 +816,8 @@ abstract class MemorySegmentIndexInput extends IndexInput return super.readLong(pos + offset); } - public MemorySegment segmentSliceOrNull(long pos, int len) throws IOException { + @Override + public MemorySegment segmentSliceOrNull(long pos, long len) throws IOException { if (pos + len > length) { throw handlePositionalIOOBE(null, "segmentSliceOrNull", pos); } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java index 3c201ce6835..93ad6b3b6b2 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java @@ -64,11 +64,13 @@ public class TestForDeltaUtil extends LuceneTestCase { { // decode IndexInput in = d.openInput("test.bin", IOContext.READONCE); - final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(new ForUtil()); + ForUtil forUtil = new ForUtil(); + PostingIndexInput postingIn = new PostingIndexInput(in, forUtil); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); for (int i = 0; i < iterations; ++i) { long base = 0; final long[] restored = new long[ForUtil.BLOCK_SIZE]; - forDeltaUtil.decodeAndPrefixSum(in, base, restored); + forDeltaUtil.decodeAndPrefixSum(postingIn, base, restored); final long[] expected = new long[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { expected[j] = values[i * ForUtil.BLOCK_SIZE + j]; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java index 114a9d0415c..dec60fc6762 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java @@ -69,12 +69,13 @@ public class TestForUtil extends LuceneTestCase { { // decode IndexInput in = d.openInput("test.bin", IOContext.READONCE); - final ForUtil forUtil = new ForUtil(); + ForUtil forUtil = new ForUtil(); + PostingIndexInput postingIn = new PostingIndexInput(in, forUtil); for (int i = 0; i < iterations; ++i) { final int bitsPerValue = in.readByte(); final long currentFilePointer = in.getFilePointer(); final long[] restored = new long[ForUtil.BLOCK_SIZE]; - forUtil.decode(bitsPerValue, in, restored); + postingIn.decode(bitsPerValue, restored); int[] ints = new int[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { ints[j] = Math.toIntExact(restored[j]); diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java index 08fec7a3a33..d185aff7645 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java @@ -38,15 +38,17 @@ public class TestPForUtil extends LuceneTestCase { final Directory d = new ByteBuffersDirectory(); final long endPointer = encodeTestData(iterations, values, d); + ForUtil forUtil = new ForUtil(); IndexInput in = d.openInput("test.bin", IOContext.READONCE); - final PForUtil pforUtil = new PForUtil(new ForUtil()); + PostingIndexInput postingIn = new PostingIndexInput(in, forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); for (int i = 0; i < iterations; ++i) { if (random().nextInt(5) == 0) { pforUtil.skip(in); continue; } final long[] restored = new long[ForUtil.BLOCK_SIZE]; - pforUtil.decode(in, restored); + pforUtil.decode(postingIn, restored); int[] ints = new int[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { ints[j] = Math.toIntExact(restored[j]); diff --git a/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java b/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java new file mode 100644 index 00000000000..64d1c23930e --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java @@ -0,0 +1,81 @@ +/* + * 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.internal.vectorization; + +import org.apache.lucene.codecs.lucene912.ForUtil; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.MMapDirectory; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestPostingDecodingUtil extends LuceneTestCase { + + public void testDuelSplitLongs() throws Exception { + final int iterations = atLeast(100); + + try (Directory dir = new MMapDirectory(createTempDir())) { + try (IndexOutput out = dir.createOutput("tests.bin", IOContext.DEFAULT)) { + out.writeInt(random().nextInt()); + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + out.writeLong(random().nextInt()); + } + } + VectorizationProvider vectorizationProvider = VectorizationProvider.lookup(true); + try (IndexInput in = dir.openInput("tests.bin", IOContext.DEFAULT)) { + long[] expectedB = new long[ForUtil.BLOCK_SIZE]; + long[] expectedC = new long[ForUtil.BLOCK_SIZE]; + long[] actualB = new long[ForUtil.BLOCK_SIZE]; + long[] actualC = new long[ForUtil.BLOCK_SIZE]; + for (int iter = 0; iter < iterations; ++iter) { + // Initialize arrays with random content. + for (int i = 0; i < expectedB.length; ++i) { + expectedB[i] = random().nextLong(); + actualB[i] = expectedB[i]; + expectedC[i] = random().nextLong(); + actualC[i] = expectedC[i]; + } + int count = TestUtil.nextInt(random(), 1, 64); + int bShift = TestUtil.nextInt(random(), 1, 31); + long bMask = random().nextLong(); + int cIndex = random().nextInt(64); + long cMask = random().nextLong(); + long startFP = random().nextInt(4); + + // Work on a slice that has just the right number of bytes to make the test fail with an + // index-out-of-bounds in case the implementation reads more than the allowed number of + // padding bytes. + IndexInput slice = in.slice("test", 0, startFP + count * Long.BYTES); + + PostingDecodingUtil defaultUtil = new DefaultPostingDecodingUtil(slice); + PostingDecodingUtil optimizedUtil = vectorizationProvider.newPostingDecodingUtil(slice); + + slice.seek(startFP); + defaultUtil.splitLongs(count, expectedB, bShift, bMask, expectedC, cIndex, cMask); + long expectedEndFP = slice.getFilePointer(); + slice.seek(startFP); + optimizedUtil.splitLongs(count, actualB, bShift, bMask, actualC, cIndex, cMask); + assertEquals(expectedEndFP, slice.getFilePointer()); + assertArrayEquals(expectedB, actualB); + assertArrayEquals(expectedC, actualC); + } + } + } + } +} From 2b0c6cdcbc4b478251dce6c0452a496d8791965e Mon Sep 17 00:00:00 2001 From: Michael Sokolov Date: Wed, 14 Aug 2024 08:55:21 -0400 Subject: [PATCH 65/66] Call HnswGraphBuilder.getCompletedGraph() in 94/95 back-compat writers (#13654) --- .../backward_codecs/lucene94/Lucene94HnswVectorsWriter.java | 4 ++-- .../backward_codecs/lucene95/Lucene95HnswVectorsWriter.java | 4 ++-- .../java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java | 3 ++- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java index 9726a3b19e8..a4f26953bd8 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsWriter.java @@ -693,9 +693,9 @@ public final class Lucene94HnswVectorsWriter extends KnnVectorsWriter { lastDocID = docID; } - OnHeapHnswGraph getGraph() { + OnHeapHnswGraph getGraph() throws IOException { if (vectors.size() > 0) { - return hnswGraphBuilder.getGraph(); + return hnswGraphBuilder.getCompletedGraph(); } else { return null; } diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsWriter.java index c74d34fb9ad..a60b5408a51 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsWriter.java @@ -732,9 +732,9 @@ public final class Lucene95HnswVectorsWriter extends KnnVectorsWriter { lastDocID = docID; } - OnHeapHnswGraph getGraph() { + OnHeapHnswGraph getGraph() throws IOException { if (vectors.size() > 0) { - return hnswGraphBuilder.getGraph(); + return hnswGraphBuilder.getCompletedGraph(); } else { return null; } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java index 8072bae462e..2ad8e9c2a16 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java @@ -412,6 +412,7 @@ public class HnswGraphBuilder implements HnswBuilder { } void finish() throws IOException { + // System.out.println("finish " + frozen); connectComponents(); frozen = true; } @@ -438,7 +439,7 @@ public class HnswGraphBuilder implements HnswBuilder { maxConn *= 2; } List components = HnswUtil.components(hnsw, level, notFullyConnected, maxConn); - // System.out.println("HnswGraphBuilder.connectComponents " + components); + // System.out.println("HnswGraphBuilder.connectComponents level=" + level + ": " + components); boolean result = true; if (components.size() > 1) { // connect other components to the largest one From 97e30fa487d2133fc43d7180d34c5605e3805926 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Wed, 14 Aug 2024 10:08:08 -0700 Subject: [PATCH 66/66] Revert changes to IndexSearcher brought in by GH#13568 (#13656) Note that this results in some silly code duplication so this is meant to be a more temporary fix while we better understand the root cause of the regression. --- .../apache/lucene/search/IndexSearcher.java | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 6842d214236..a8f8d14c24f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -630,12 +630,45 @@ public class IndexSearcher { */ public T search(Query query, CollectorManager collectorManager) throws IOException { - CollectorOwner collectorOwner = new CollectorOwner<>(collectorManager); - final C firstCollector = collectorOwner.newCollector(); + final C firstCollector = collectorManager.newCollector(); query = rewrite(query, firstCollector.scoreMode().needsScores()); final Weight weight = createWeight(query, firstCollector.scoreMode(), 1); - search(weight, collectorOwner, firstCollector); - return collectorOwner.getResult(); + return search(weight, collectorManager, firstCollector); + } + + private T search( + Weight weight, CollectorManager collectorManager, C firstCollector) throws IOException { + final LeafSlice[] leafSlices = getSlices(); + if (leafSlices.length == 0) { + // there are no segments, nothing to offload to the executor, but we do need to call reduce to + // create some kind of empty result + assert leafContexts.isEmpty(); + return collectorManager.reduce(Collections.singletonList(firstCollector)); + } else { + final List collectors = new ArrayList<>(leafSlices.length); + collectors.add(firstCollector); + final ScoreMode scoreMode = firstCollector.scoreMode(); + for (int i = 1; i < leafSlices.length; ++i) { + final C collector = collectorManager.newCollector(); + collectors.add(collector); + if (scoreMode != collector.scoreMode()) { + throw new IllegalStateException( + "CollectorManager does not always produce collectors with the same score mode"); + } + } + final List> listTasks = new ArrayList<>(leafSlices.length); + for (int i = 0; i < leafSlices.length; ++i) { + final LeafReaderContext[] leaves = leafSlices[i].leaves; + final C collector = collectors.get(i); + listTasks.add( + () -> { + search(Arrays.asList(leaves), weight, collector); + return collector; + }); + } + List results = taskExecutor.invokeAll(listTasks); + return collectorManager.reduce(results); + } } /**