Re: [PR] read MSB VLong in new way [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1765808689 Hi @jpountz , Thanks a lot for the suggestion! > another option could be to encode the number of supplementary bytes using unary coding (like UTF8). This is a great idea that probably makes `readMSBVLong` more faster ! FYI, the direction I'm considering is that it's not "decoding the MSB VLong" that causes this regression, but "how the MSB VLong changes the FST structure": * For LSB VLong output, most/all of the bytes are stored in single arc. * For MSB VLong output, bytes are spilitted into many arcs for prefix sharing. So we will need to more `Outputs#read` and `Outputs#add` on for `MSBVLong` to get the whole output. Here is a comparing of call times between LSB VLong (before #12631) and MSB VLong (after #12631) https://bytedance.feishu.cn/sheets/Yp2Zs5ngphNWEHtZHUic8WlZnUf"; data-importRangeRawData-range="'Sheet1'!A1:D3"> | LSB VLong | MSB VLong | diff -- | -- | -- | -- Outputs#read times | 116097 | 149803 | 29.03% Outputs#add times | 144 | 111568 | 77377.78% Unfortunately, `ByteSequenceOutputs#add` and `ByteSequenceOutputs#read` always need to construct new `BytesRef` objects, not efficient enough. This patch tried to speed up `ByteSequenceOutputs#add` a bit , getting the tiny improvement [mentioned above](https://github.com/apache/lucene/pull/12661#issuecomment-1764814636). But we are still seeing the regression there because `add` still needed while origin patch just ignore the NO_OUTPUT arcs. So i'm not very sure the optimization of the decoding output can resolve the regression as it does not look like the bottleneck to me, but I'd like to give a try if you still think it is worth :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
jpountz commented on code in PR #12682: URL: https://github.com/apache/lucene/pull/12682#discussion_r1361646368 ## lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java: ## @@ -266,7 +265,7 @@ public float score() throws IOException { score += optScorer.score(); } -return score; +return (float) score; Review Comment: Actually your change doesn't help here since this sums up two floats at most and summing up two floats is already guaranteed to be as accurate as a float can be. Let's revert changes on this file? ## lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java: ## @@ -504,9 +504,9 @@ public TFIDFScorer(float boost, Explanation idf, float[] normTable) { @Override public float score(float freq, long norm) { - final float raw = tf(freq) * queryWeight; // compute tf(f)*weight + final double raw = tf(freq) * queryWeight; // compute tf(f)*weight float normValue = normTable[(int) (norm & 0xFF)]; - return raw * normValue; // normalize for field + return (float) (raw * normValue); // normalize for field Review Comment: Likewise here, float multiplication is already guaranteed to give a result that is as accurate as a float can give. One could argue that we could get more accuracy by casting into a double before multiplying in the first multiplication, ie. `final double raw = (double) tf(freq) * queryWeight;`. But I don't think we should do it as similarity scores are a bit fuzzy by nature, and this would very unlikely improve ranking effectiveness. The main reason why we cast into doubles when summing up scores in not really to get better accuracy, but more so that the other in which clauses are evaluated doesn't have an impact on the final score. Let's revert changes on this file as well? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] read MSB VLong in new way [lucene]
jpountz commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1765890646 Oh your explanation makes sense, and I agree with you that a more efficient encoding would unlikely help conterbalance the fact that more arcs need to be read per output. So this looks like a hard search/space trade-off: we either get fast reads or good compression but we can't get both? I wonder if extending the `Outputs` class directly would help, instead of storing data in an opaque `byte[]`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
shubhamvishu commented on code in PR #12682: URL: https://github.com/apache/lucene/pull/12682#discussion_r1361736510 ## lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java: ## @@ -266,7 +265,7 @@ public float score() throws IOException { score += optScorer.score(); } -return score; +return (float) score; Review Comment: Makes sense to me! I think in that case we should remove the TODO as well? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
shubhamvishu commented on code in PR #12682: URL: https://github.com/apache/lucene/pull/12682#discussion_r1361737240 ## lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java: ## @@ -504,9 +504,9 @@ public TFIDFScorer(float boost, Explanation idf, float[] normTable) { @Override public float score(float freq, long norm) { - final float raw = tf(freq) * queryWeight; // compute tf(f)*weight + final double raw = tf(freq) * queryWeight; // compute tf(f)*weight float normValue = normTable[(int) (norm & 0xFF)]; - return raw * normValue; // normalize for field + return (float) (raw * normValue); // normalize for field Review Comment: I see. Thanks for clarifying! I'll revert changes to this file too. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
jpountz commented on code in PR #12682: URL: https://github.com/apache/lucene/pull/12682#discussion_r1361739665 ## lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java: ## @@ -266,7 +265,7 @@ public float score() throws IOException { score += optScorer.score(); } -return score; +return (float) score; Review Comment: I think that the TODO still makes sense, it refers to BS1 being able to handle a mix of MUST and SHOULD clauses. If it happened, then it could have more than 2 clauses so casting into a double would make sense. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] read MSB VLong in new way [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1765964640 > I wonder if extending the Outputs class directly would help, instead of storing data in an opaque byte[]? Yes ,The reuse is exactly what `Outputs` wants to do ! (see this [todo](https://github.com/apache/lucene/blob/6dac2f7afce483a35cb97e4f2022693f4168c496/lucene/core/src/java/org/apache/lucene/util/fst/Outputs.java#L35)). However, It seems that by contract the `Outputs#add` should not modify the input objects (this is not declared in java doc but assumed by callers). I was thinking about adding a new method to `Outputs` like following, WDYT? ``` public Adder adder(T reuse) { return new Adder<>() { T result; @Override public void add(T newOne) { result = Outputs.this.add(result, newOne); } @Override public T result() { return result; } }; } public interface Adder { void add(T newOne); T result(); } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
jpountz commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1765975335 If I read correctly, this query ends up calling `LeafReader#searchNearestNeighbors` with k=Integer.MAX_VALUE, which will not only run in O(maxDoc) time but also use O(maxDoc) memory. I don't think we should do this. In my opinion, there are two options: either we force this query to take a `k` parameter and make it only return the top `k` nearest neighbors that are also within the radius. Or we make it always run in "exact" mode with a two-phase iterator that performs the similary check in `TwoPhaseIterator#matches()`. We'd then need to prefix this query with `Slow` like other queries that work similarly. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
shubhamvishu commented on PR #12682: URL: https://github.com/apache/lucene/pull/12682#issuecomment-1765985160 Thanks @jpountz for the review! I have addressed the comments in the new revision. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Scorer should sum up scores into a double [lucene]
shubhamvishu commented on code in PR #12682: URL: https://github.com/apache/lucene/pull/12682#discussion_r1361773569 ## lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java: ## @@ -266,7 +265,7 @@ public float score() throws IOException { score += optScorer.score(); } -return score; +return (float) score; Review Comment: Sure, I'll keep it then -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add a merge policy wrapper that performs recursive graph bisection on merge. [lucene]
jpountz commented on code in PR #12622: URL: https://github.com/apache/lucene/pull/12622#discussion_r1361783707 ## lucene/core/src/java/org/apache/lucene/index/IndexWriter.java: ## @@ -5144,20 +5145,71 @@ public int length() { } mergeReaders.add(wrappedReader); } + + MergeState.DocMap[] reorderDocMaps = null; + if (config.getIndexSort() == null) { +// Create a merged view of the input segments. This effectively does the merge. +CodecReader mergedView = SlowCompositeCodecReaderWrapper.wrap(mergeReaders); +Sorter.DocMap docMap = merge.reorder(mergedView, directory); +if (docMap != null) { + reorderDocMaps = new MergeState.DocMap[mergeReaders.size()]; + int docBase = 0; + int i = 0; + for (CodecReader reader : mergeReaders) { +final int finalDocBase = docBase; +reorderDocMaps[i] = +new MergeState.DocMap() { + @Override + public int get(int docID) { +Objects.checkIndex(docID, reader.maxDoc()); +return docMap.oldToNew(finalDocBase + docID); + } +}; +i++; +docBase += reader.maxDoc(); + } + // This makes merging more expensive as it disables some bulk merging optimizations, so + // only do this if a non-null DocMap is returned. + mergeReaders = + Collections.singletonList(SortingCodecReader.wrap(mergedView, docMap, null)); +} + } + final SegmentMerger merger = new SegmentMerger( mergeReaders, merge.info.info, infoStream, dirWrapper, globalFieldNumberMap, context); merge.info.setSoftDelCount(Math.toIntExact(softDeleteCount.get())); merge.checkAborted(); + MergeState mergeState = merger.mergeState; + MergeState.DocMap[] docMaps; + if (reorderDocMaps == null) { +docMaps = mergeState.docMaps; + } else { +assert mergeState.docMaps.length == 1; +MergeState.DocMap compactionDocMap = mergeState.docMaps[0]; +docMaps = new MergeState.DocMap[reorderDocMaps.length]; +for (int i = 0; i < docMaps.length; ++i) { + MergeState.DocMap reorderDocMap = reorderDocMaps[i]; + docMaps[i] = + new MergeState.DocMap() { +@Override +public int get(int docID) { + int reorderedDocId = reorderDocMap.get(docID); Review Comment: We could try to combine the two doc maps. I did it this way to try to keep the reordering logic and the merging logic as independent as possible. On the one hand, we have the reordering logic that works on a merged view of the input segments and doesn't care about deletes. On the other hand, we have the merge logic that computes the mapping between doc IDs in input segments and doc IDs in the merged segment (often just compacting deletes, ie. if index sorting is not enabled). If we wanted to better combine these two things, we'd need to either ignore the `MergeState`'s doc maps or somehow make `MergeState` aware of the reordering. Today `MergeState` is completely unaware of the reordering, from its perspective it just needs to run a singleton merge (single input codec reader) and its only job is to reclaim deletes. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add a merge policy wrapper that performs recursive graph bisection on merge. [lucene]
jpountz commented on code in PR #12622: URL: https://github.com/apache/lucene/pull/12622#discussion_r1361793823 ## lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java: ## @@ -468,7 +468,11 @@ public void checkIntegrity() throws IOException { @Override public PointValues getValues(String field) throws IOException { -return new SortingPointValues(delegate.getValues(field), docMap); +PointValues values = delegate.getValues(field); Review Comment: Thanks for catching, it should not change as file formats should never be called on fields whose info say the field is not indexed. I fixed the slow composite reader wrapper instead. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add a merge policy wrapper that performs recursive graph bisection on merge. [lucene]
jpountz commented on code in PR #12622: URL: https://github.com/apache/lucene/pull/12622#discussion_r1361798124 ## lucene/core/src/java/org/apache/lucene/index/SlowCompositeCodecReaderWrapper.java: ## @@ -0,0 +1,998 @@ +/* + * 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 java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.codecs.PointsReader; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues; +import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.Version; + +/** + * A merged {@link CodecReader} view of multiple {@link CodecReader}. This view is primarily + * targeted at merging, not searching. + */ +final class SlowCompositeCodecReaderWrapper extends CodecReader { + + static CodecReader wrap(List readers) throws IOException { +switch (readers.size()) { + case 0: +throw new IllegalArgumentException("Must take at least one reader, got 0"); + case 1: +return readers.get(0); + default: +return new SlowCompositeCodecReaderWrapper(readers); +} + } + + private final LeafMetaData meta; + private final CodecReader[] codecReaders; + private final int[] docStarts; + private final FieldInfos fieldInfos; + private final Bits liveDocs; + + private SlowCompositeCodecReaderWrapper(List codecReaders) throws IOException { +this.codecReaders = codecReaders.toArray(CodecReader[]::new); +docStarts = new int[codecReaders.size() + 1]; +int i = 0; +int docStart = 0; +for (CodecReader reader : codecReaders) { + i++; + docStart += reader.maxDoc(); + docStarts[i] = docStart; +} +int majorVersion = -1; +Version minVersion = null; +for (CodecReader reader : codecReaders) { + LeafMetaData readerMeta = reader.getMetaData(); + if (majorVersion == -1) { +majorVersion = readerMeta.getCreatedVersionMajor(); + } else if (majorVersion != readerMeta.getCreatedVersionMajor()) { +throw new IllegalArgumentException( +"Cannot combine leaf readers created with different major versions"); + } + if (minVersion == null) { +minVersion = readerMeta.getMinVersion(); + } else if (minVersion.onOrAfter(readerMeta.getMinVersion())) { +minVersion = readerMeta.getMinVersion(); + } +} +meta = new LeafMetaData(majorVersion, minVersion, null); +MultiReader multiReader = new MultiReader(codecReaders.toArray(CodecReader[]::new)); +fieldInfos = FieldInfos.getMergedFieldInfos(multiReader); +liveDocs = MultiBits.getLiveDocs(multiReader); + } + + @Override + public StoredFieldsReader getFieldsReader() { +StoredFieldsReader[] readers = +Arrays.stream(codecReaders) +.map(CodecReader::getFieldsReader) +.toArray(StoredFieldsReader[]::new); +return new SlowCompositeStoredFieldsReaderWrapper(readers, docStarts); + } + + // Remap FieldInfos to make sure consumers only see field infos from the composite reader, not + // from individual leaves + private FieldInfo remap(FieldInfo info) { +return fieldInfos.fieldInfo(info.name); + } + + private class SlowCompositeStoredFieldsReaderWrapper extends StoredFieldsReader { + +private final StoredFieldsReader[] readers; +private final int[] docStarts; + +SlowCompositeStoredFieldsReaderWrapper(StoredFieldsReader[] readers, int[] docStarts) { + this.readers = readers; + this.docSt
Re: [PR] Add a merge policy wrapper that performs recursive graph bisection on merge. [lucene]
jpountz commented on code in PR #12622: URL: https://github.com/apache/lucene/pull/12622#discussion_r1361799042 ## lucene/core/src/java/org/apache/lucene/index/SlowCompositeCodecReaderWrapper.java: ## @@ -0,0 +1,998 @@ +/* + * 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 java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.codecs.PointsReader; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues; +import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.KnnCollector; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.Version; + +/** + * A merged {@link CodecReader} view of multiple {@link CodecReader}. This view is primarily + * targeted at merging, not searching. + */ +final class SlowCompositeCodecReaderWrapper extends CodecReader { + + static CodecReader wrap(List readers) throws IOException { +switch (readers.size()) { + case 0: +throw new IllegalArgumentException("Must take at least one reader, got 0"); + case 1: +return readers.get(0); + default: +return new SlowCompositeCodecReaderWrapper(readers); +} + } + + private final LeafMetaData meta; + private final CodecReader[] codecReaders; + private final int[] docStarts; + private final FieldInfos fieldInfos; + private final Bits liveDocs; + + private SlowCompositeCodecReaderWrapper(List codecReaders) throws IOException { +this.codecReaders = codecReaders.toArray(CodecReader[]::new); +docStarts = new int[codecReaders.size() + 1]; +int i = 0; +int docStart = 0; +for (CodecReader reader : codecReaders) { + i++; + docStart += reader.maxDoc(); + docStarts[i] = docStart; +} +int majorVersion = -1; +Version minVersion = null; +for (CodecReader reader : codecReaders) { + LeafMetaData readerMeta = reader.getMetaData(); + if (majorVersion == -1) { +majorVersion = readerMeta.getCreatedVersionMajor(); + } else if (majorVersion != readerMeta.getCreatedVersionMajor()) { +throw new IllegalArgumentException( +"Cannot combine leaf readers created with different major versions"); + } + if (minVersion == null) { +minVersion = readerMeta.getMinVersion(); + } else if (minVersion.onOrAfter(readerMeta.getMinVersion())) { +minVersion = readerMeta.getMinVersion(); + } +} +meta = new LeafMetaData(majorVersion, minVersion, null); +MultiReader multiReader = new MultiReader(codecReaders.toArray(CodecReader[]::new)); +fieldInfos = FieldInfos.getMergedFieldInfos(multiReader); +liveDocs = MultiBits.getLiveDocs(multiReader); + } + + @Override + public StoredFieldsReader getFieldsReader() { +StoredFieldsReader[] readers = +Arrays.stream(codecReaders) +.map(CodecReader::getFieldsReader) +.toArray(StoredFieldsReader[]::new); +return new SlowCompositeStoredFieldsReaderWrapper(readers, docStarts); + } + + // Remap FieldInfos to make sure consumers only see field infos from the composite reader, not + // from individual leaves + private FieldInfo remap(FieldInfo info) { +return fieldInfos.fieldInfo(info.name); + } + + private class SlowCompositeStoredFieldsReaderWrapper extends StoredFieldsReader { + +private final StoredFieldsReader[] readers; +private final int[] docStarts; + +SlowCompositeStoredFieldsReaderWrapper(StoredFieldsReader[] readers, int[] docStarts) { + this.readers = readers; + this.docSt
Re: [PR] Add a merge policy wrapper that performs recursive graph bisection on merge. [lucene]
jpountz commented on code in PR #12622: URL: https://github.com/apache/lucene/pull/12622#discussion_r1361802385 ## lucene/core/src/java/org/apache/lucene/index/IndexWriter.java: ## @@ -5144,20 +5145,71 @@ public int length() { } mergeReaders.add(wrappedReader); } + + MergeState.DocMap[] reorderDocMaps = null; + if (config.getIndexSort() == null) { +// Create a merged view of the input segments. This effectively does the merge. +CodecReader mergedView = SlowCompositeCodecReaderWrapper.wrap(mergeReaders); +Sorter.DocMap docMap = merge.reorder(mergedView, directory); +if (docMap != null) { + reorderDocMaps = new MergeState.DocMap[mergeReaders.size()]; + int docBase = 0; + int i = 0; + for (CodecReader reader : mergeReaders) { +final int finalDocBase = docBase; +reorderDocMaps[i] = +new MergeState.DocMap() { + @Override + public int get(int docID) { +Objects.checkIndex(docID, reader.maxDoc()); +return docMap.oldToNew(finalDocBase + docID); + } +}; +i++; +docBase += reader.maxDoc(); + } + // This makes merging more expensive as it disables some bulk merging optimizations, so + // only do this if a non-null DocMap is returned. + mergeReaders = + Collections.singletonList(SortingCodecReader.wrap(mergedView, docMap, null)); +} + } + final SegmentMerger merger = new SegmentMerger( mergeReaders, merge.info.info, infoStream, dirWrapper, globalFieldNumberMap, context); merge.info.setSoftDelCount(Math.toIntExact(softDeleteCount.get())); merge.checkAborted(); + MergeState mergeState = merger.mergeState; + MergeState.DocMap[] docMaps; + if (reorderDocMaps == null) { +docMaps = mergeState.docMaps; + } else { +assert mergeState.docMaps.length == 1; +MergeState.DocMap compactionDocMap = mergeState.docMaps[0]; Review Comment: The above line already has an assertion, I'll add a comment too. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
mikemccand commented on code in PR #12633: URL: https://github.com/apache/lucene/pull/12633#discussion_r1361812236 ## lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java: ## @@ -17,50 +17,80 @@ package org.apache.lucene.util.fst; import java.io.IOException; -import org.apache.lucene.util.packed.PackedInts; -import org.apache.lucene.util.packed.PagedGrowableWriter; +import java.util.Arrays; +import java.util.Locale; // Used to dedup states (lookup already-frozen states) final class NodeHash { - private PagedGrowableWriter table; - private long count; - private long mask; + // primary table + private long[] table; + + // how many nodes are stored in the primary table; when this gets full, we discard tableOld and move primary to it + private int count; + private int promoteCount; + private int missCount; + private int hitCount; + + // fallback table. if we fallback and find the frozen node here, we promote it to primary table, for a simplistic LRU behaviour + private long[] fallbackTable; + + private final int mask; private final FST fst; private final FST.Arc scratchArc = new FST.Arc<>(); private final FST.BytesReader in; - public NodeHash(FST fst, FST.BytesReader in) { -table = new PagedGrowableWriter(16, 1 << 27, 8, PackedInts.COMPACT); -mask = 15; + public NodeHash(FST fst, int tableSize, FST.BytesReader in) { +if (tableSize < 4) { + // 2 is a power of 2, but does not work because the rehash logic (at 2/3 capacity) becomes over-quantized, and the hash + // table becomes 100% full before moving to fallback, and then looking up an entry in 100% full hash table spins forever + throw new IllegalArgumentException("tableSize must at least 4; got: " + tableSize); +} +mask = tableSize - 1; +if ((mask & tableSize) != 0) { + throw new IllegalArgumentException("tableSize must be a power of 2; got: " + tableSize); +} +table = new long[tableSize]; +fallbackTable = new long[tableSize]; this.fst = fst; this.in = in; } + /** Compares an unfrozen node (UnCompiledNode) with a frozen node at byte location address (long), returning + * true if they are equal. */ private boolean nodesEqual(FSTCompiler.UnCompiledNode node, long address) throws IOException { fst.readFirstRealTargetArc(address, scratchArc, in); Review Comment: I think we should do that (fully decouple `NodeHash` from the FST's `byte[]`) as a follow-on issue? This change is tricky enough :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Use radix sort to speed up the sorting of terms in TermInSetQuery [lucene]
gf2121 merged PR #12587: URL: https://github.com/apache/lucene/pull/12587 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
mikemccand commented on code in PR #12633: URL: https://github.com/apache/lucene/pull/12633#discussion_r1361814551 ## lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java: ## @@ -99,31 +87,23 @@ public class FSTCompiler { * tuning and tweaking, see {@link Builder}. */ public FSTCompiler(FST.INPUT_TYPE inputType, Outputs outputs) { Review Comment: Yeah, that's a good point -- I'm not sure why we have a public ctor on this class. I'll see if I can remove it, maybe as follow-on PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
mikemccand commented on PR #12633: URL: https://github.com/apache/lucene/pull/12633#issuecomment-1766041651 > > With the PR, you unfortunately cannot easily say "give me a minimal FST at all costs", like you can with main today. You'd have to keep trying larger and larger NodeHash sizes until the final FST size gets no smaller. > > If we replace `long[]` with a growable array that inits from small and grows smoothly to `nodeHashSize`, can we just pass a big nodeHashSize (e.g. 1L << 63) or a constant `NO_LIMIT` to get a minimal FST ? I love this idea! This way the hash grows to consume only as much RAM as needed, up until the specified limit, at which point it begins pruning to cap the RAM usage at the limit. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Optimize outputs accumulating as MSB VLong outputs sharing more output prefix [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1766048357 Hi @mikemccand , it would be great if you can take a look too :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Use MergeSorter in StableStringSorter [lucene]
gf2121 merged PR #12652: URL: https://github.com/apache/lucene/pull/12652 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
mikemccand commented on PR #12633: URL: https://github.com/apache/lucene/pull/12633#issuecomment-1766082689 Thanks for the suggestions @dungba88! I took the approach you suggested, with a few more pushed commits just now. Despite the increase in `nocommit`s I think this is actually close! I like this new approach: * It uses the same mutable packed blocked growable (in size and bpv) writer thingy (`PagedGrowableWriter`) that `NodeHash` uses on `main` * But now the FSTCompiler (and its Builder) take an option to set a limit on the size (count of number of suffix entries) of the `NodeHash`. I plan to change this to a `ramMB` limit instead * If you set a massive limit (`Long.MAX_VALUE`) then every suffix is stored (as compactly as on `main` today) and you get a minimal FST. * If you set a lower limit and the `NodeHash` hits it, it will begin pruning the LRU suffixes, and you get a somewhat compressed FST. The larger the limit, the more RAM used, and the closer to minimal your FST is. I tested again on all terms from `wikimediumall` index: |NodeHash size|FST (mb)|RAM (mb)|Build time (sec)| |-|||| |4|585.8|0.0|110.0| |8|587.0|0.0|74.7| |16|586.3|0.0|60.1| |32|583.7|0.0|52.5| |64|580.4|0.0|46.5| |128|575.9|0.0|44.0| |256|568.0|0.0|42.6| |512|556.6|0.0|41.8| |1024|543.2|0.0|42.4| |2048|529.3|0.0|40.9| |4096|515.2|0.0|41.0| |8192|501.5|0.1|40.8| |16384|488.2|0.1|40.3| |32768|474.0|0.2|41.5| |65536|453.0|0.5|42.0| |131072|439.0|0.9|41.6|
Re: [PR] Remove over-counting of deleted terms [lucene]
gf2121 merged PR #12586: URL: https://github.com/apache/lucene/pull/12586 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Speed up TestIndexOrDocValuesQuery. [lucene]
jpountz merged PR #12672: URL: https://github.com/apache/lucene/pull/12672 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Specialize `BlockImpactsDocsEnum#nextDoc()`. [lucene]
jpountz merged PR #12670: URL: https://github.com/apache/lucene/pull/12670 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362016333 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** + * Will scalar quantize float vectors into `int8` byte values. This is a lossy transformation. + * Scalar quantization works by first calculating the quantiles of the float vector values. The + * quantiles are calculated using the configured quantile/confidence interval. The [minQuantile, + * maxQuantile] are then used to scale the values into the range [0, 127] and bucketed into the Review Comment: @jmazanec15 Java doesn't have unsigned bytes & handling signed values in the math is exceptionally difficult. We only use one bucket of possible values, the impact in recall is minimal (if measurable at all). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Fix lazy decoding of frequencies in `BlockImpactsDocsEnum`. [lucene]
jpountz commented on PR #12668: URL: https://github.com/apache/lucene/pull/12668#issuecomment-1766342540 Even though the speedup is less pronounced than in the above luceneutil run, there seems to be an actual speedup in nightly benchmarks for boolean queries. E.g. the last 3 data points of [`AndHighMed`](http://people.apache.org/~mikemccand/lucenebench/AndHighMed.html) are the highest throughput numbers ever recorded (though they're not much faster than on the previous days). I pushed an annotation that should show up tomorrow. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] analysis-stempel incorrect tokens generation for numbers [LUCENE-10290] [lucene]
tomsquest commented on issue #11326: URL: https://github.com/apache/lucene/issues/11326#issuecomment-1766389365 This issue occurred to us also, and not only for numbers. Actually, token finishing by `1` will be stemmed! ``` GET _analyze { "tokenizer": "standard", "filter": [ "polish_stem" ], "text": "ha1" } // OUTPUT { "tokens": [ { "token": "hć", // < stemmed! "start_offset": 0, "end_offset": 3, "type": "", "position": 0 } ] } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Reduce collection operations when minShouldMatch == 0. [lucene]
jpountz commented on PR #12602: URL: https://github.com/apache/lucene/pull/12602#issuecomment-1766428044 I would be surprisid if this change would yield a noticeable speedup? Does it? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
mayya-sharipova commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362208743 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ## @@ -0,0 +1,1149 @@ +/* + * 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.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene95.OffHeapByteVectorValues; +import org.apache.lucene.codecs.lucene95.OffHeapFloatVectorValues; +import org.apache.lucene.codecs.lucene95.OrdToDocDISIReaderConfiguration; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { +this.M = M; +this.beamWidth = beamWidth; +segmentWriteState = state; +String metaFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + +String vectorDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, +state.segmentSuffix, +Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + +String indexDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
mayya-sharipova commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362208743 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ## @@ -0,0 +1,1149 @@ +/* + * 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.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene95.OffHeapByteVectorValues; +import org.apache.lucene.codecs.lucene95.OffHeapFloatVectorValues; +import org.apache.lucene.codecs.lucene95.OrdToDocDISIReaderConfiguration; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { +this.M = M; +this.beamWidth = beamWidth; +segmentWriteState = state; +String metaFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + +String vectorDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, +state.segmentSuffix, +Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + +String indexDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo
Re: [PR] [BROKEN, for reference only] concurrent hnsw [lucene]
msokolov commented on code in PR #12683: URL: https://github.com/apache/lucene/pull/12683#discussion_r1362245604 ## lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraph.java: ## @@ -59,11 +60,26 @@ protected HnswGraph() {} * * @param level level of the graph * @param target ordinal of a node in the graph, must be ≥ 0 and < {@link - * FloatVectorValues#size()}. + * HnswGraph#size()}. */ public abstract void seek(int level, int target) throws IOException; - /** Returns the number of nodes in the graph */ + public interface NeighborIterator extends Closeable { +int nextNeighbor() throws IOException; +int size(); + } + + /** + * Retrieve an iterator over the neighbors of a graph node, acquiring a read lock. Closing the iterator + * releases the lock. + * @param level level of the graph + * @param node ordinal of a node in the graph, must be ≥ 0 and < {@link HnswGraph#size()}. + */ + public NeighborIterator lockNeighbors(int level, int node) throws IOException { +throw new UnsupportedOperationException("concurrent access not implemented"); Review Comment: ah true. I think the impl for Lucene95HnswVectorsReader.OffHeapHnswGraph can be moved up to here -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766741617 > If I read correctly, this query ends up calling LeafReader#searchNearestNeighbors with k=Integer.MAX_VALUE No, we're calling the [new API](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/LeafReader.java#L329) (from [here](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/RnnFloatVectorQuery.java#L57)) with a custom [`RnnCollector`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/RnnCollector.java#L28) that performs score-based HNSW searches (as opposed to the [old API](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/LeafReader.java#L246-L247) that performs `topK`-based searches with `k=Integer.MAX_VALUE`) The `Integer.MAX_VALUE` passed [here](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java#L38) is just used in two places: [`#exactSearch`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L174) (to instantiate a priority queue of size `k`) and [`#mergeLeafResults`](https://github.com/apache/lucene/blob/a384967b5ea631de940e327f4483888c80d09611/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L216) (to request for the best-scoring `k` hits across all segment results). We're overriding both functions in our implementation of `AbstractRnnVectorQuery` (because we do not want to limit to `topK` results) I think you're worried that we'll end up performing brute-force KNN on all documents in the segment, and *then* retain vectors above the threshold? What we instead aim to do is: starting from the entry node in the last level of HNSW graphs, we keep visiting candidates as long as they are above the `traversalThreshold`, all the while adding nodes above the `resultThreshold` as accepted results This is not necessarily slower than normal HNSW searches, provided the `traversalThreshold` is chosen suitably -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362432970 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** + * Will scalar quantize float vectors into `int8` byte values. This is a lossy transformation. + * Scalar quantization works by first calculating the quantiles of the float vector values. The + * quantiles are calculated using the configured quantile/confidence interval. The [minQuantile, + * maxQuantile] are then used to scale the values into the range [0, 127] and bucketed into the + * nearest byte values. + * + * How Scalar Quantization Works Review Comment: @jmazanec15 the configured quantile/confidence interval is used to calculate the upper and lower quantiles for the accepted values. I think earlier I mention this in the comment. ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** + * Will scalar quantize float vectors into `int8` byte values. This is a lossy transformation. + * Scalar quantization works by first calculating the quantiles of the float vector values. The + * quantiles are calculated using the configured quantile/confidence interval. The [minQuantile, + * maxQuantile] are then used to scale the values into the range [0, 127] and bucketed into the + * nearest byte values. + * + * How Scalar Quantization Works + * + * The basic mathematical equations behind this are fairly straight forward. Given a float vector + * `v` and a quantile `q` we can calculate the quantiles of the vector values [minQuantile, + * maxQuantile]. + * + * + * float = (maxQuantile - minQuantile)/127 * byte + minQuantile + * byte = (float - minQuantile) * 127/(maxQuantile - minQuantile) + * + * + * This then means to multiply two float values together (e.g. dot_product) we can do the + * following: + * + * + * float1 * float2 = (byte1 * (maxQuantile - minQuantile)/127 + minQuantile) * (byte2 * (maxQuantile - minQuantile)/127 + minQuantile) Review Comment: ++ -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362437535 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** + * Will scalar quantize float vectors into `int8` byte values. This is a lossy transformation. + * Scalar quantization works by first calculating the quantiles of the float vector values. The + * quantiles are calculated using the configured quantile/confidence interval. The [minQuantile, + * maxQuantile] are then used to scale the values into the range [0, 127] and bucketed into the Review Comment: > one bucket Should be `one bit`, its true we have half as many buckets to play with. But we are getting really good numbers without bothering with it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]
benwtrent commented on code in PR #12657: URL: https://github.com/apache/lucene/pull/12657#discussion_r1362456237 ## lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java: ## @@ -0,0 +1,98 @@ +/* + * 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 org.apache.lucene.util.BitSet; + +/** + * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for + * merging HnswGraphs from multiple segments. + * + * @lucene.experimental + */ +public final class InitializedHnswGraphBuilder extends HnswGraphBuilder { + + /** + * Create a new HnswGraphBuilder that is initialized with the provided HnswGraph. + * + * @param scorerSupplier the scorer to use for vectors + * @param M the number of connections to keep per node + * @param beamWidth the number of nodes to explore in the search + * @param seed the seed for the random number generator + * @param initializerGraph the graph to initialize the new graph builder + * @param newOrdMap a mapping from the old node ordinal to the new node ordinal + * @param initializedNodes a bitset of nodes that are already initialized in the initializerGraph + * @return a new HnswGraphBuilder that is initialized with the provided HnswGraph + * @throws IOException when reading the graph fails + */ + public static InitializedHnswGraphBuilder fromGraph( + RandomVectorScorerSupplier scorerSupplier, + int M, + int beamWidth, + long seed, + HnswGraph initializerGraph, + int[] newOrdMap, + BitSet initializedNodes) + throws IOException { +OnHeapHnswGraph hnsw = new OnHeapHnswGraph(M); +for (int level = 0; level < initializerGraph.numLevels(); level++) { Review Comment: updated! Thanks for the callout! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]
benwtrent commented on code in PR #12657: URL: https://github.com/apache/lucene/pull/12657#discussion_r1362457390 ## lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java: ## @@ -0,0 +1,197 @@ +/* + * 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.Map; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.CollectionUtil; +import org.apache.lucene.util.FixedBitSet; + +/** + * This selects the biggest Hnsw graph from the provided merge state and initializes a new + * HnswGraphBuilder with that graph as a starting point. + * + * @lucene.experimental + */ +public class IncrementalHnswGraphMerger { + + private KnnVectorsReader initReader; + private MergeState.DocMap initDocMap; + private int initGraphSize; + private final FieldInfo fieldInfo; + private final RandomVectorScorerSupplier scorerSupplier; + private final int M; + private final int beamWidth; + + /** + * @param fieldInfo FieldInfo for the field being merged + */ + public IncrementalHnswGraphMerger( + FieldInfo fieldInfo, RandomVectorScorerSupplier scorerSupplier, int M, int beamWidth) { +this.fieldInfo = fieldInfo; +this.scorerSupplier = scorerSupplier; +this.M = M; +this.beamWidth = beamWidth; + } + + /** + * Adds a reader to the graph merger if it meets the following criteria: 1. Does not contain any + * deleted docs 2. Is a HnswGraphProvider/PerFieldKnnVectorReader 3. Has the most docs of any + * previous reader that met the above criteria + * + * @param reader KnnVectorsReader to add to the merger + * @param docMap MergeState.DocMap for the reader + * @param liveDocs Bits representing live docs, can be null + * @return this + * @throws IOException If an error occurs while reading from the merge state + */ + public IncrementalHnswGraphMerger addReader( + KnnVectorsReader reader, MergeState.DocMap docMap, Bits liveDocs) throws IOException { +KnnVectorsReader currKnnVectorsReader = reader; +if (reader instanceof PerFieldKnnVectorsFormat.FieldsReader candidateReader) { + currKnnVectorsReader = candidateReader.getFieldReader(fieldInfo.name); +} + +if (!(currKnnVectorsReader instanceof HnswGraphProvider) || !allMatch(liveDocs)) { + return this; +} + +int candidateVectorCount = 0; +switch (fieldInfo.getVectorEncoding()) { + case BYTE -> { +ByteVectorValues byteVectorValues = +currKnnVectorsReader.getByteVectorValues(fieldInfo.name); +if (byteVectorValues == null) { + return this; +} +candidateVectorCount = byteVectorValues.size(); + } + case FLOAT32 -> { +FloatVectorValues vectorValues = currKnnVectorsReader.getFloatVectorValues(fieldInfo.name); +if (vectorValues == null) { + return this; +} +candidateVectorCount = vectorValues.size(); + } +} +if (candidateVectorCount > initGraphSize) { + initReader = currKnnVectorsReader; + initDocMap = docMap; + initGraphSize = candidateVectorCount; +} +return this; + } + + /** + * Builds a new HnswGraphBuilder using the biggest graph from the merge state as a starting point. + * If no valid readers were added to the merge state, a new graph is created. + * + * @param mergeState MergeState for the merge + * @return HnswGraphBuilder + * @throws IOException If an error occurs while reading from the merge state + */ + public HnswGraphBuilder createBuilder(MergeState mergeState) throws IOException {
Re: [PR] Add support for radius-based vector searches [lucene]
jpountz commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766795903 Thanks for explaining, I had overlooked how the `Integer.MAX_VALUE` was used indeed. I'm still interested in figuring out if we can have stronger guarantees on the worst-case memory usage that this query could have (I believe noting prevents this list from growing unbounded? if the threshold is high?). E.g. could we abort the approximate search if the list maintained by the `RnnCollector` grows too large, and fall back to an exact search that is based on a `TwoPhaseIterator` instead of eagerly collecting all matches into a list? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on code in PR #12679: URL: https://github.com/apache/lucene/pull/12679#discussion_r1362472568 ## lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java: ## @@ -0,0 +1,105 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.LeafReaderContext; + +/** + * Search for all (approximate) vectors within a radius using the {@link RnnCollector}. + * + * @lucene.experimental + */ +abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery { + private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS; Review Comment: No strong opinions here, I had just kept it similar to `AbstractKnnVectorQuery` Changed now.. ## lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java: ## @@ -0,0 +1,105 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.LeafReaderContext; + +/** + * Search for all (approximate) vectors within a radius using the {@link RnnCollector}. + * + * @lucene.experimental + */ +abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery { + private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS; + + protected final float traversalThreshold, resultThreshold; Review Comment: Added -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on code in PR #12679: URL: https://github.com/apache/lucene/pull/12679#discussion_r1362475149 ## lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java: ## @@ -0,0 +1,105 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.LeafReaderContext; + +/** + * Search for all (approximate) vectors within a radius using the {@link RnnCollector}. + * + * @lucene.experimental + */ +abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery { + private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS; + + protected final float traversalThreshold, resultThreshold; + + public AbstractRnnVectorQuery( + String field, float traversalThreshold, float resultThreshold, Query filter) { +super(field, Integer.MAX_VALUE, filter); +assert traversalThreshold <= resultThreshold; +this.traversalThreshold = traversalThreshold; +this.resultThreshold = resultThreshold; + } + + @Override + protected TopDocs exactSearch(LeafReaderContext context, DocIdSetIterator acceptIterator) + throws IOException { +@SuppressWarnings("resource") +FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field); +if (fi == null || fi.getVectorDimension() == 0) { + // The field does not exist or does not index vectors + return NO_RESULTS; +} + +VectorScorer vectorScorer = createVectorScorer(context, fi); +List scoreDocList = new ArrayList<>(); + +int doc; +while ((doc = acceptIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { + boolean advanced = vectorScorer.advanceExact(doc); + assert advanced; + + float score = vectorScorer.score(); + if (score >= resultThreshold) { +scoreDocList.add(new ScoreDoc(doc, score)); + } +} + +TotalHits totalHits = new TotalHits(acceptIterator.cost(), TotalHits.Relation.EQUAL_TO); +return new TopDocs(totalHits, scoreDocList.toArray(ScoreDoc[]::new)); + } + + @Override + // Segment-level results are not sorted (because we do not want to maintain the topK), just + // concatenate them + protected TopDocs mergeLeafResults(TopDocs[] perLeafResults) { +long value = 0; +TotalHits.Relation relation = TotalHits.Relation.EQUAL_TO; +List scoreDocList = new ArrayList<>(); + +for (TopDocs topDocs : perLeafResults) { + value += topDocs.totalHits.value; + if (topDocs.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) { +relation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; + } + scoreDocList.addAll(List.of(topDocs.scoreDocs)); +} + +return new TopDocs(new TotalHits(value, relation), scoreDocList.toArray(ScoreDoc[]::new)); + } + + @Override + public boolean equals(Object o) { +if (this == o) return true; +if (o == null || getClass() != o.getClass()) return false; +if (!super.equals(o)) return false; Review Comment: This was the default `equals` method generated by the IDE. Made it more in line with other classes now -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on code in PR #12679: URL: https://github.com/apache/lucene/pull/12679#discussion_r1362474206 ## lucene/core/src/java/org/apache/lucene/search/AbstractRnnVectorQuery.java: ## @@ -0,0 +1,105 @@ +/* + * 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.List; +import java.util.Objects; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.LeafReaderContext; + +/** + * Search for all (approximate) vectors within a radius using the {@link RnnCollector}. + * + * @lucene.experimental + */ +abstract class AbstractRnnVectorQuery extends AbstractKnnVectorQuery { + private static final TopDocs NO_RESULTS = TopDocsCollector.EMPTY_TOPDOCS; + + protected final float traversalThreshold, resultThreshold; + + public AbstractRnnVectorQuery( + String field, float traversalThreshold, float resultThreshold, Query filter) { +super(field, Integer.MAX_VALUE, filter); +assert traversalThreshold <= resultThreshold; Review Comment: Agreed! `traversalThreshold > resultThreshold` does not make sense, and we want to warn users about this more strongly. Changed now -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on code in PR #12679: URL: https://github.com/apache/lucene/pull/12679#discussion_r1362476143 ## lucene/core/src/java/org/apache/lucene/search/RnnCollector.java: ## @@ -0,0 +1,71 @@ +/* + * 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.util.ArrayList; +import java.util.List; + +/** + * A collector that performs radius-based vector searches. All vectors within an outer radius are + * traversed, and those within an inner radius are collected. + * + * @lucene.experimental + */ +public class RnnCollector extends AbstractKnnCollector { + private final float traversalThreshold, resultThreshold; + private final List scoreDocList; + + /** + * Performs radius-based vector searches. + * + * @param traversalThreshold similarity score corresponding to outer radius of graph traversal. + * @param resultThreshold similarity score corresponding to inner radius of result collection. + * @param visitLimit limit of graph nodes to visit. + */ + public RnnCollector(float traversalThreshold, float resultThreshold, long visitLimit) { +super(Integer.MAX_VALUE, visitLimit); +assert traversalThreshold <= resultThreshold; +this.traversalThreshold = traversalThreshold; +this.resultThreshold = resultThreshold; +this.scoreDocList = new ArrayList<>(); + } + + @Override + public boolean collect(int docId, float similarity) { +if (similarity >= resultThreshold) { + return scoreDocList.add(new ScoreDoc(docId, similarity)); +} +return false; + } + + @Override + public float minCompetitiveSimilarity() { +return traversalThreshold; + } + + @Override + // This does not return results in a sorted order to prevent unnecessary calculations (because we + // do not want to maintain the topK) Review Comment: Moved to more suitable places -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766834111 Thanks for the review @shubhamvishu! Addressed some of the comments above > Is it right to call it a radius-based search here? I think of it as finding all results within a high-dimensional circle / sphere / equivalent, and the radius-based search seems to capture the essence. Although "threshold-based search" may be more appropriate (since radius is tied to Euclidean Distance, and may not be easy to relate with Cosine Similarity or Dot Product) No strong opinions here, looking for others' thoughts as well on more appropriate naming.. > The RnnFloatVectorQuery and RnnByteVectorQuery are almost the same The problem here is that we'll have to generalize many other (unrelated to this change) internal classes. I'll keep this to a separate issue -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Fix SynonymQuery equals implementation [lucene]
mingshl commented on PR #12260: URL: https://github.com/apache/lucene/pull/12260#issuecomment-1766881156 Thank you! @mkhludnev -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Extract the hnsw graph merging from being part of the vector writer [lucene]
benwtrent merged PR #12657: URL: https://github.com/apache/lucene/pull/12657 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Optimize outputs accumulating as MSB VLong outputs sharing more output prefix [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1766958252 An idea comes to me that maybe we do not really need to do combine all these `BytesRef`s to a single `BytesRef`, we can just build a `DataInput` over these `BytesRef`s to read. Luckily, only the encoded MSB VLong could probably be splitted into several arcs in FST while floor data, which is more sensitive to reading performance, is guaranteed stored continuously in single arc. With this optimization, i'm seeing PKLookup almost gets back to original speed: > Baseline: **before** https://github.com/apache/lucene/pull/12631; > Candidate: this patch; > Luceneutil Random seed = 0 ``` TaskQPS baseline StdDevQPS my_modified_version StdDevPct diff p-value Fuzzy2 39.46 (1.3%) 37.93 (1.0%) -3.9% ( -6% - -1%) 0.000 Respell 33.92 (2.3%) 33.04 (1.4%) -2.6% ( -6% -1%) 0.000 Wildcard 60.14 (2.6%) 58.73 (2.3%) -2.3% ( -7% -2%) 0.004 Fuzzy1 54.73 (1.5%) 53.62 (1.2%) -2.0% ( -4% -0%) 0.000 PKLookup 111.96 (2.3%) 111.45 (2.3%) -0.5% ( -5% -4%) 0.556 ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] TaskExecutor to cancel all tasks on exception [lucene]
javanna opened a new pull request, #12689: URL: https://github.com/apache/lucene/pull/12689 When operations are parallelized, like query rewrite, or search, or createWeight, one of the tasks may throw an exception. In that case we wait for all tasks to be completed before re-throwing the exception that were caught. Tasks that were not started when the exception is captured though can be safely skipped. Ideally we would also cancel ongoing tasks but I left that for another time. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
benwtrent commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766983182 > I think of it as finding all results within a high-dimensional circle / sphere / equivalent, dot-product, cosine, etc. don't really follow that same idea as you point out. I would prefer something like `VectorSimilarityQuery` or something. > E.g. could we abort the approximate search if the list maintained by the RnnCollector grows too large, and fall back to an exact search that is based on a TwoPhaseIterator instead of eagerly collecting all matches into a list? I agree with @jpountz concerns. The topDocs collector gets a replay of the matched documents. We should put sane limits here and prevent folks from getting 100,000s of matches (int & float value arrays) via approximate search. It seems like having a huge number like that could cause issues. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] TaskExecutor to cancel all tasks on exception [lucene]
javanna commented on code in PR #12689: URL: https://github.com/apache/lucene/pull/12689#discussion_r1362620375 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -64,64 +67,124 @@ public final class TaskExecutor { * @param the return type of the task execution */ public List invokeAll(Collection> callables) throws IOException { -List> tasks = new ArrayList<>(callables.size()); -boolean runOnCallerThread = numberOfRunningTasksInCurrentThread.get() > 0; -for (Callable callable : callables) { - Task task = new Task<>(callable); - tasks.add(task); - if (runOnCallerThread) { -task.run(); - } else { -executor.execute(task); +TaskGroup taskGroup = new TaskGroup<>(callables); +return taskGroup.invokeAll(executor); + } + + /** + * Holds all the sub-tasks that a certain operation gets split into as it gets parallelized and + * exposes the ability to invoke such tasks and wait for them all to complete their execution and + * provide their results. Ensures that each task does not get parallelized further: this is + * important to avoid a deadlock in situations where one executor thread waits on other executor + * threads to complete before it can progress. This happens in situations where for instance + * {@link Query#createWeight(IndexSearcher, ScoreMode, float)} is called as part of searching each + * slice, like {@link TopFieldCollector#populateScores(ScoreDoc[], IndexSearcher, Query)} does. + * Additionally, if one task throws an exception, all other tasks from the same group are + * cancelled, to avoid needless computation as their results would not be exposed anyways. Creates + * one {@link FutureTask} for each {@link Callable} provided + * + * @param the return type of all the callables + */ + private static final class TaskGroup { Review Comment: the diff is hard to read. The task is the same as before, with the addition of some FutureTask methods override: - setException to handle the exception and cancel all tasks on exception. we can't simply catch the exception on run, because FutureTask#run does not throw it. We could have wrapped the callable, but I prefer leaving the original callable unchanged and overrideing FutureTask behavior instead - cancel to deal with task cancellations: the original behaviour would be for future.get to throw a cancellation exception when called on a cancelled task, while still leaving the task running. We instead want to wait for all tasks to be completed before returning. I introduced the private TaskGroup abstraction to group all tasks and primarily to make the `cancelAll` method available against a final list. This is to address the dependency between the creation of the FutureTask, which needs to cancel all tasks on exception, yet the tasks list is populated only once all tasks have been created. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] TaskExecutor to cancel all tasks on exception [lucene]
javanna commented on code in PR #12689: URL: https://github.com/apache/lucene/pull/12689#discussion_r1362621063 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -64,64 +67,124 @@ public final class TaskExecutor { * @param the return type of the task execution */ public List invokeAll(Collection> callables) throws IOException { -List> tasks = new ArrayList<>(callables.size()); -boolean runOnCallerThread = numberOfRunningTasksInCurrentThread.get() > 0; -for (Callable callable : callables) { - Task task = new Task<>(callable); - tasks.add(task); - if (runOnCallerThread) { -task.run(); - } else { -executor.execute(task); +TaskGroup taskGroup = new TaskGroup<>(callables); +return taskGroup.invokeAll(executor); + } + + /** + * Holds all the sub-tasks that a certain operation gets split into as it gets parallelized and + * exposes the ability to invoke such tasks and wait for them all to complete their execution and + * provide their results. Ensures that each task does not get parallelized further: this is + * important to avoid a deadlock in situations where one executor thread waits on other executor + * threads to complete before it can progress. This happens in situations where for instance + * {@link Query#createWeight(IndexSearcher, ScoreMode, float)} is called as part of searching each + * slice, like {@link TopFieldCollector#populateScores(ScoreDoc[], IndexSearcher, Query)} does. + * Additionally, if one task throws an exception, all other tasks from the same group are + * cancelled, to avoid needless computation as their results would not be exposed anyways. Creates + * one {@link FutureTask} for each {@link Callable} provided + * + * @param the return type of all the callables + */ + private static final class TaskGroup { +private final Collection> futures; + +TaskGroup(Collection> callables) { + List> tasks = new ArrayList<>(callables.size()); + for (Callable callable : callables) { +tasks.add(createTask(callable)); } + this.futures = Collections.unmodifiableCollection(tasks); } -Throwable exc = null; -final List results = new ArrayList<>(); -for (Future future : tasks) { - try { -results.add(future.get()); - } catch (InterruptedException e) { -var newException = new ThreadInterruptedException(e); -if (exc == null) { - exc = newException; -} else { - exc.addSuppressed(newException); +private FutureTask createTask(Callable callable) { + AtomicBoolean started = new AtomicBoolean(false); + return new FutureTask<>(callable) { +@Override +public void run() { + if (started.compareAndSet(false, true)) { +try { + Integer counter = numberOfRunningTasksInCurrentThread.get(); + numberOfRunningTasksInCurrentThread.set(counter + 1); + super.run(); +} finally { + Integer counter = numberOfRunningTasksInCurrentThread.get(); + numberOfRunningTasksInCurrentThread.set(counter - 1); +} + } else { +// task is cancelled hence it has no results to return. That's fine: they would be +// ignored anyway. +set(null); + } +} + +@Override +protected void setException(Throwable t) { + cancelAll(); + super.setException(t); } - } catch (ExecutionException e) { -if (exc == null) { - exc = e.getCause(); + +@Override +public boolean cancel(boolean mayInterruptIfRunning) { + assert mayInterruptIfRunning == false; + /* + Future#get (called in invokeAll) throws CancellationException for a cancelled task when invoked but leaves the task running. + We rather want to make sure that invokeAll does not leave any running tasks behind when it returns. + Overriding cancel ensures that tasks that are already started will complete normally once cancelled, and Future#get will + wait for them to finish instead of throwing CancellationException. Tasks that are cancelled before they are started won't start. + */ + return started.compareAndSet(false, true); +} + }; +} + +List invokeAll(Executor executor) throws IOException { + boolean runOnCallerThread = numberOfRunningTasksInCurrentThread.get() > 0; + for (Runnable runnable : futures) { +if (runOnCallerThread) { + runnable.run(); } else { - exc.addSuppressed(e.getCause()); + executor.execute(runnable); } } + Throwable exc = null; + List results = new ArrayList<>(futures.size()); + for (Future future : futures) { +try { + results.add(future.get()); +} c
Re: [PR] TaskExecutor to cancel all tasks on exception [lucene]
javanna commented on code in PR #12689: URL: https://github.com/apache/lucene/pull/12689#discussion_r1362621950 ## lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java: ## @@ -43,7 +47,8 @@ public class TestTaskExecutor extends LuceneTestCase { public static void createExecutor() { executorService = Executors.newFixedThreadPool( -1, new NamedThreadFactory(TestTaskExecutor.class.getSimpleName())); +random().nextBoolean() ? 1 : 2, +new NamedThreadFactory(TestTaskExecutor.class.getSimpleName())); Review Comment: we have a test that verifies the case for multiple exceptions, which becomes impossible to reproduce with a single threaded pool. That's why I randomized the pool size between 1 and 2. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] TaskExecutor to cancel all tasks on exception [lucene]
javanna commented on code in PR #12689: URL: https://github.com/apache/lucene/pull/12689#discussion_r1362621950 ## lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java: ## @@ -43,7 +47,8 @@ public class TestTaskExecutor extends LuceneTestCase { public static void createExecutor() { executorService = Executors.newFixedThreadPool( -1, new NamedThreadFactory(TestTaskExecutor.class.getSimpleName())); +random().nextBoolean() ? 1 : 2, +new NamedThreadFactory(TestTaskExecutor.class.getSimpleName())); Review Comment: we have a test that verifies the case for multiple exceptions, which becomes impossible to reproduce with a single threaded pool, because the first exception makes us cancel all other tasks, hence you can't have another exception thrown. That's why I randomized the pool size between 1 and 2. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1766995337 ### Benchmarks Using the vector file from https://home.apache.org/~sokolov/enwiki-20120502-lines-1k-100d.vec (enwiki dataset, unit vectors, 100 dimensions) The setup was 1M doc vectors in a single HNSW graph with `DOT_PRODUCT` similarity, and 10K query vectors The baseline for the new objective is "all vectors above a score threshold" (as opposed to the best-scoring `topK` vectors in the current system) for a given query and is used to compute recall in all subsequent runs.. Here are some statistics for the result counts in the new baseline: | threshold | mean | stdDev| min | p25 | p50 | p75| p90| p99| max| | - | | - | --- | --- | | -- | -- | -- | -- | | 0.95 | 71877.73 | 109177.23 | 0 | 222 | 7436 | 116567 | 259135 | 388113 | 483330 | | 0.96 | 32155.63 | 57183.83 | 0 | 30 | 3524 | 36143 | 120700 | 235038 | 342959 | | 0.97 | 8865.48 | 19006.24 | 0 | 1 | 816 | 5483 | 29966 | 92433 | 174163 | | 0.98 | 1010.10 | 2423.03 | 0 | 0 | 46 | 873| 3234 | 12175 | 40163 | | 0.99 | 136.47 | 465.91| 0 | 0 | 0| 2 | 77 | 2296 | 2494 | This is used to get an estimate of query - result count distribution for various `threshold` values, and also gauge the corresponding `topK` to use for comparison with the new radius-based vector search API Here we will benchmark the new API against a high `topK` (+ filtering out results below the threshold after HNSW search) ### K-NN Search (current system) | maxConn | beamWidth | topK | threshold | mean | numVisited | latency | recall | | --- | - | | - | -- | -- | --- | -- | | 16 | 100 | 500 | 0.99 | 46.39 | 4086 | 1.465 | 0.34 | | 16 | 100 | 1000 | 0.99 | 83.92 | 6890 | 2.600 | 0.61 | | 16 | 100 | 2000 | 0.99 | 129.56 | 11727 | 4.746 | 0.95 | | 16 | 200 | 500 | 0.99 | 46.39 | 4504 | 1.535 | 0.34 | | 16 | 200 | 1000 | 0.99 | 83.92 | 7564 | 2.759 | 0.61 | | 16 | 200 | 2000 | 0.99 | 129.56 | 12805 | 5.007 | 0.95 | | 32 | 100 | 500 | 0.99 | 46.39 | 4940 | 1.644 | 0.34 | | 32 | 100 | 1000 | 0.99 | 83.92 | 8271 | 2.944 | 0.61 | | 32 | 100 | 2000 | 0.99 | 129.56 | 13937 | 5.335 | 0.95 | | 32 | 200 | 500 | 0.99 | 46.39 | 5654 | 1.890 | 0.34 | | 32 | 200 | 1000 | 0.99 | 83.92 | 9401 | 3.320 | 0.61 | | 32 | 200 | 2000 | 0.99 | 129.56 | 15707 | 5.987 | 0.95 | | 64 | 100 | 500 | 0.99 | 46.39 | 5241 | 1.736 | 0.34 | | 64 | 100 | 1000 | 0.99 | 83.92 | 8766 | 3.091 | 0.61 | | 64 | 100 | 2000 | 0.99 | 129.56 | 14736 | 5.567 | 0.95 | | 64 | 200 | 500 | 0.99 | 46.39 | 6095 | 1.992 | 0.34 | | 64 | 200 | 1000 | 0.99 | 83.92 | 10119 | 3.535 | 0.61 | | 64 | 200 | 2000 | 0.99 | 129.56 | 16852 | 6.365 | 0.95 | ### R-NN Search (new system) | maxConn | beamWidth | traversalThreshold | threshold | mean | numVisited | latency | recall | | --- | - | -- | - | -- | -- | --- | -- | | 16 | 100 | 0.99 | 0.99 | 94.03 | 256 | 0.129 | 0.69 | | 16 | 100 | 0.98 | 0.99 | 95.18 | 5171 | 2.062 | 0.70 | | 16 | 200 | 0.99 | 0.99 | 89.96 | 263 | 0.119 | 0.66 | | 16 | 200 | 0.98 | 0.99 | 91.09 | 5497 | 2.207 | 0.67 | | 32 | 100 | 0.99 | 0.99 | 109.17 | 295 | 0.135 | 0.80 | | 32 | 100 | 0.98 | 0.99 | 110.89 | 6529 | 2.580 | 0.81 | | 32 | 200 | 0.99 | 0.99 | 108.97 | 313 | 0.142 | 0.80 | | 32 | 200 | 0.98 | 0.99 | 110.55 | 7145 | 2.861 | 0.81 | | 64 | 100 | 0.99 | 0.99 | 133.61 | 314 | 0.152 | 0.98 | | 64 | 100 | 0.98 | 0.99 | 135.74 | 7033 | 2.765 | 0.99 | | 64 | 200 | 0.99 | 0.99 | 133.84 | 333 | 0.163 | 0.98 | | 64 | 200 | 0.98 | 0.99 | 135.96 | 7833 | 3.121 | 1.00 | - `mea
Re: [PR] Add support for radius-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1767022898 > stronger guarantees on the worst-case memory usage Totally agreed @jpountz! It is very easy to go wrong in the new API, specially if the user passes a low threshold (high radius -> low threshold). As we can see from benchmarks above, the number of nodes to visit may jump very fast with slight reduction in the `traversalThreshold` (`mean` column of first table) > fall back to an exact search that is based on a `TwoPhaseIterator` This makes sense to me.. Something like a lazy-loading iterator, where we perform vector comparisons and determine whether a doc matches on `#advance`? > something like `VectorSimilarityQuery` I like this, thanks for the suggestion @benwtrent! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362661760 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java: ## @@ -0,0 +1,1149 @@ +/* + * 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.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat.DIRECT_MONOTONIC_BLOCK_SHIFT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.HnswGraphProvider; +import org.apache.lucene.codecs.KnnFieldVectorsWriter; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.lucene95.OffHeapByteVectorValues; +import org.apache.lucene.codecs.lucene95.OffHeapFloatVectorValues; +import org.apache.lucene.codecs.lucene95.OrdToDocDISIReaderConfiguration; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.ByteVectorValues; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier; +import org.apache.lucene.util.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraph.NodesIterator; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.NeighborArray; +import org.apache.lucene.util.hnsw.OnHeapHnswGraph; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.packed.DirectMonotonicWriter; + +/** + * Writes vector values and knn graphs to index segments. + * + * @lucene.experimental + */ +public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter { + + private final SegmentWriteState segmentWriteState; + private final IndexOutput meta, vectorData, quantizedVectorData, vectorIndex; + private final int M; + private final int beamWidth; + private final Lucene99ScalarQuantizedVectorsWriter quantizedVectorsWriter; + + private final List> fields = new ArrayList<>(); + private boolean finished; + + Lucene99HnswVectorsWriter( + SegmentWriteState state, + int M, + int beamWidth, + Lucene99ScalarQuantizedVectorsFormat quantizedVectorsFormat) + throws IOException { +this.M = M; +this.beamWidth = beamWidth; +segmentWriteState = state; +String metaFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, state.segmentSuffix, Lucene99HnswVectorsFormat.META_EXTENSION); + +String vectorDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name, +state.segmentSuffix, +Lucene99HnswVectorsFormat.VECTOR_DATA_EXTENSION); + +String indexDataFileName = +IndexFileNames.segmentFileName( +state.segmentInfo.name,
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362664506 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java: ## @@ -0,0 +1,782 @@ +/* + * 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.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; +import static org.apache.lucene.util.RamUsageEstimator.shallowSizeOfInstance; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.List; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.DocIDMerger; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.VectorUtil; +import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier; +import org.apache.lucene.util.hnsw.RandomVectorScorer; + +/** + * Writes quantized vector values and metadata to index segments. + * + * @lucene.experimental + */ +public final class Lucene99ScalarQuantizedVectorsWriter implements Accountable { + + private static final long BASE_RAM_BYTES_USED = + shallowSizeOfInstance(Lucene99ScalarQuantizedVectorsWriter.class); + + private static final float QUANTIZATION_RECOMPUTE_LIMIT = 32; + private final IndexOutput quantizedVectorData; + private final Float quantile; + private boolean finished; + + Lucene99ScalarQuantizedVectorsWriter(IndexOutput quantizedVectorData, Float quantile) { +this.quantile = quantile; +this.quantizedVectorData = quantizedVectorData; + } + + QuantizationVectorWriter addField(FieldInfo fieldInfo, InfoStream infoStream) { +if (fieldInfo.getVectorEncoding() != VectorEncoding.FLOAT32) { + throw new IllegalArgumentException( + "Only float32 vector fields are supported for quantization"); +} +float quantile = +this.quantile == null +? calculateDefaultQuantile(fieldInfo.getVectorDimension()) +: this.quantile; +if (infoStream.isEnabled(QUANTIZED_VECTOR_COMPONENT)) { + infoStream.message( + QUANTIZED_VECTOR_COMPONENT, + "quantizing field=" + + fieldInfo.name + + " dimension=" + + fieldInfo.getVectorDimension() + + " quantile=" + + quantile); +} +return QuantizationVectorWriter.create(fieldInfo, quantile, infoStream); + } + + long[] flush( + Sorter.DocMap sortMap, QuantizationVectorWriter field, DocsWithFieldSet docsWithField) + throws IOException { +field.finish(); +return sortMap == null ? writeField(field) : writeSortingField(field, sortMap, docsWithField); + } + + void finish() throws IOException { +if (finished) { + throw new IllegalStateException("already finished"); +} +finished = true; +if (quantizedVectorData != null) { + CodecUtil.writeFooter(quantizedVectorData); +} + } + + private long[] writeField(QuantizationVectorWriter fieldData) throws IOException { +long
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362665321 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java: ## @@ -0,0 +1,782 @@ +/* + * 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.lucene99; + +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.QUANTIZED_VECTOR_COMPONENT; +import static org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat.calculateDefaultQuantile; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; +import static org.apache.lucene.util.RamUsageEstimator.shallowSizeOfInstance; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.List; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.KnnVectorsWriter; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.index.DocIDMerger; +import org.apache.lucene.index.DocsWithFieldSet; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.MergeState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Sorter; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ScalarQuantizer; +import org.apache.lucene.util.VectorUtil; +import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier; +import org.apache.lucene.util.hnsw.RandomVectorScorer; + +/** + * Writes quantized vector values and metadata to index segments. + * + * @lucene.experimental + */ +public final class Lucene99ScalarQuantizedVectorsWriter implements Accountable { + + private static final long BASE_RAM_BYTES_USED = + shallowSizeOfInstance(Lucene99ScalarQuantizedVectorsWriter.class); + + private static final float QUANTIZATION_RECOMPUTE_LIMIT = 32; + private final IndexOutput quantizedVectorData; + private final Float quantile; + private boolean finished; + + Lucene99ScalarQuantizedVectorsWriter(IndexOutput quantizedVectorData, Float quantile) { +this.quantile = quantile; +this.quantizedVectorData = quantizedVectorData; + } + + QuantizationVectorWriter addField(FieldInfo fieldInfo, InfoStream infoStream) { +if (fieldInfo.getVectorEncoding() != VectorEncoding.FLOAT32) { + throw new IllegalArgumentException( + "Only float32 vector fields are supported for quantization"); +} +float quantile = +this.quantile == null +? calculateDefaultQuantile(fieldInfo.getVectorDimension()) +: this.quantile; +if (infoStream.isEnabled(QUANTIZED_VECTOR_COMPONENT)) { + infoStream.message( + QUANTIZED_VECTOR_COMPONENT, + "quantizing field=" + + fieldInfo.name + + " dimension=" + + fieldInfo.getVectorDimension() + + " quantile=" + + quantile); +} +return QuantizationVectorWriter.create(fieldInfo, quantile, infoStream); + } + + long[] flush( + Sorter.DocMap sortMap, QuantizationVectorWriter field, DocsWithFieldSet docsWithField) + throws IOException { +field.finish(); +return sortMap == null ? writeField(field) : writeSortingField(field, sortMap, docsWithField); + } + + void finish() throws IOException { +if (finished) { + throw new IllegalStateException("already finished"); +} +finished = true; +if (quantizedVectorData != null) { + CodecUtil.writeFooter(quantizedVectorData); +} + } + + private long[] writeField(QuantizationVectorWriter fieldData) throws IOException { +long
Re: [I] [DISCUSS] Should there be a threshold-based vector search API? [lucene]
kaivalnp commented on issue #12579: URL: https://github.com/apache/lucene/issues/12579#issuecomment-1767112899 > one other thing to think about is https://weaviate.io/blog/weaviate-1-20-release#autocut Interesting! They [seem to](https://github.com/weaviate/weaviate/blob/c382dcbe6ff05dfadf1e2799fdb4ade2d60aead8/entities/autocut/autocut.go#L14) make "cuts" when there is an abnormal jump in scores. An "abnormal jump" is defined when the score of the next candidate does not match the gradual decrease in scores (for example: if we have collected 10 vectors with a max and min score of 1.0 and 0.8, the "step" is basically 0.02 per vector, so we make a "cut" if the next best candidate is below a score of 0.78) Might be a good next step to simplify the API! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1362725464 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.util; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** Will scalar quantize float vectors into `int8` byte values */ +public class ScalarQuantizer { + + public static final int SCALAR_QUANTIZATION_SAMPLE_SIZE = 25_000; + + private final float alpha; + private final float scale; + private final float minQuantile, maxQuantile, configuredQuantile; + + /** + * @param minQuantile the lower quantile of the distribution + * @param maxQuantile the upper quantile of the distribution + * @param configuredQuantile The configured quantile/confidence interval used to calculate the + * quantiles. + */ + public ScalarQuantizer(float minQuantile, float maxQuantile, float configuredQuantile) { +assert maxQuantile >= maxQuantile; +this.minQuantile = minQuantile; +this.maxQuantile = maxQuantile; +this.scale = 127f / (maxQuantile - minQuantile); +this.alpha = (maxQuantile - minQuantile) / 127f; +this.configuredQuantile = configuredQuantile; + } + + /** + * Quantize a float vector into a byte vector + * + * @param src the source vector + * @param dest the destination vector + * @param similarityFunction the similarity function used to calculate the quantile + * @return the corrective offset that needs to be applied to the score + */ + public float quantize(float[] src, byte[] dest, VectorSimilarityFunction similarityFunction) { +assert src.length == dest.length; +float correctiveOffset = 0f; +for (int i = 0; i < src.length; i++) { + float v = src[i]; + float dx = Math.max(minQuantile, Math.min(maxQuantile, src[i])) - minQuantile; + float dxs = scale * dx; + float dxq = Math.round(dxs) * alpha; + correctiveOffset += minQuantile * (v - minQuantile / 2.0F) + (dx - dxq) * dxq; + dest[i] = (byte) Math.round(dxs); +} +if (similarityFunction.equals(VectorSimilarityFunction.EUCLIDEAN)) { + return 0; +} +return correctiveOffset; + } Review Comment: The corrective offset is required for good recall. > Did this improve recall in any of the benchmarks? I am unsure what you are referring to? All the benchmarks I have done have had a corrective offset. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move private static classes or functions out of DoubleValuesSource [lucene]
gsmiller commented on PR #12671: URL: https://github.com/apache/lucene/pull/12671#issuecomment-1767291109 Thanks for your further thoughts @shubhamvishu. Getting more opinions is always good, and like I said, I don't feel strongly enough about this change to block moving forward with it or anything if others think it's a helpful thing to do. My thoughts remain though that: 1. At least to me personally, I don't necessarily find any readability improvements from separating the public API from the definitions in this way. I don't think it's particularly challenging to navigate this one file (in fact, I personally prefer it in a number of ways). But again, this is just opinion. It's sort of the "if it's not broke, don't fix it" thing I suppose from my point of view. 2. In terms of future extensibility, my opinion is to wait until there is a real use-case driving the need to break these things out rather than move code around just in case we want to use it in more places some day. Again though, this is all pretty subjective, so if others like the change, let's go for it! I appreciate you thinking about it and looking for ways to improve things! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Remove direct dependency of NodeHash to FST [lucene]
dungba88 opened a new pull request, #12690: URL: https://github.com/apache/lucene/pull/12690 ### Description Follow-up of https://github.com/apache/lucene/pull/12646. NodeHash still depends on both FSTCompiler and FST. With the current method signature, one can create the NodeHash with a FST and call addNode with different FSTCompiler. This PR simplifies the signature and allow a consistent behavior. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
dungba88 commented on code in PR #12633: URL: https://github.com/apache/lucene/pull/12633#discussion_r1363098628 ## lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java: ## @@ -17,79 +17,177 @@ package org.apache.lucene.util.fst; import java.io.IOException; -import org.apache.lucene.util.packed.PackedInts; +import java.util.Arrays; +import java.util.Locale; import org.apache.lucene.util.packed.PagedGrowableWriter; +import org.apache.lucene.util.packed.PackedInts;; + +// nocommit we could gradually grow the double-barrel hash size based on size of the growing FST? hmm does not work so well since you +// failed to use the larger hash in the beginning? it's fundamentally needing two passes? + +// TODO: any way to make a reversee suffix lookup (msokolov's idea) instead of more costly hash? hmmm, though, hash is not so wasteful +// since it does not have to store value of each entry: the value is the node pointer in the FST. actually, there is much to save +// there -- we would not need any long per entry -- we'd be able to start at the FST end node and work backwards from the transitions + +// TODO: couldn't we prune natrually babck until we see a transition with an output? it's highly unlikely (mostly impossible) such suffixes +// can be shared? // Used to dedup states (lookup already-frozen states) final class NodeHash { - private PagedGrowableWriter table; - private long count; - private long mask; + // nocommit + private static final boolean DO_PRINT_HASH_RAM = true; + + // primary table -- we add nodes into this until it reaches the requested tableSizeLimit/2, then we move it to fallback + private PagedGrowableHash primaryTable; + + // how many nodes are allowed to store in both primary and fallback tables; when primary gets full (tableSizeLimit/2), we move it to the + // fallback table + private final long ramLimitBytes; + + // fallback table. if we fallback and find the frozen node here, we promote it to primary table, for a simplistic and lowish-RAM-overhead + // (compared to e.g. LinkedHashMap) LRU behaviour + private PagedGrowableHash fallbackTable; + private final FST fst; private final FST.Arc scratchArc = new FST.Arc<>(); private final FST.BytesReader in; - public NodeHash(FST fst, FST.BytesReader in) { -table = new PagedGrowableWriter(16, 1 << 27, 8, PackedInts.COMPACT); -mask = 15; + /** ramLimitMB is the max RAM we can use for recording suffixes. If we hit this limit, the least recently used suffixes are discarded, and + * the FST is no longer minimalI. Still, larger ramLimitMB will make the FST smaller (closer to minimal). */ + public NodeHash(FST fst, double ramLimitMB, FST.BytesReader in) { +if (ramLimitMB <= 0) { + throw new IllegalArgumentException("ramLimitMB must be > 0; got: " + ramLimitMB); +} +double asBytes = ramLimitMB * 1024 * 1024; +if (asBytes >= Long.MAX_VALUE) { + // quietly truncate to Long.MAX_VALUE in bytes too + ramLimitBytes = Long.MAX_VALUE; +} else { + ramLimitBytes = (long) asBytes; +} + +primaryTable = new PagedGrowableHash(); this.fst = fst; this.in = in; } - private boolean nodesEqual(FSTCompiler.UnCompiledNode node, long address) throws IOException { -fst.readFirstRealTargetArc(address, scratchArc, in); - -// Fail fast for a node with fixed length arcs. -if (scratchArc.bytesPerArc() != 0) { - if (scratchArc.nodeFlags() == FST.ARCS_FOR_BINARY_SEARCH) { -if (node.numArcs != scratchArc.numArcs()) { - return false; -} - } else { -assert scratchArc.nodeFlags() == FST.ARCS_FOR_DIRECT_ADDRESSING; -if ((node.arcs[node.numArcs - 1].label - node.arcs[0].label + 1) != scratchArc.numArcs() -|| node.numArcs != FST.Arc.BitTable.countBits(scratchArc, in)) { - return false; -} + // nocommit measure how wasteful/conflicty these hash tables are. should we improve hash function? + + private long getFallback(FSTCompiler.UnCompiledNode nodeIn, long hash) throws IOException { +if (fallbackTable == null) { + // no fallback yet (primary table is not yet large enough to swap) + return 0; +} +long pos = hash & fallbackTable.mask; +int c = 0; +while (true) { + long node = fallbackTable.get(pos); + if (node == 0) { +// not found +return 0; + } else if (nodesEqual(nodeIn, node)) { +// frozen version of this node is already here +return node; } + + // nocommit -- is this really quadratic probe? + // quadratic probe + pos = (pos + (++c)) & fallbackTable.mask; } + } -for (int arcUpto = 0; arcUpto < node.numArcs; arcUpto++) { - final FSTCompiler.Arc arc = node.arcs[arcUpto]; - if (arc.label != scratchArc.label() - || !arc.output.equals(scratchArc.output()) - || ((FSTCompiler.CompiledNode) arc.target).
Re: [I] HnwsGraph creates disconnected components [lucene]
nitirajrathore commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1767662289 I was able to run tests on wiki dataset using the luceneutils package. The [results shows](https://github.com/mikemccand/luceneutil/pull/236) that even with a single segment index and no updates, around 1% nodes gets disconnected for about 1M vectors. It would be great if someone else can have a look at the ``CheckHNSWConnectedness`` for correctness. This may or may not be an issue for different system given that this is 'approximate' nearest neighbour search. But in my opinion it is worth exploring more and if possible some fix. Next I will try to reproduce with multiple segments and try to find the cause and fix for it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Refactor ByteBlockPool so it is just a "shift/mask big array" [lucene]
iverase merged PR #12625: URL: https://github.com/apache/lucene/pull/12625 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Optimize outputs accumulating as MSB VLong outputs sharing more output prefix [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1767756956 > So this looks like a hard search/space trade-off: we either get fast reads or good compression but we can't get both? IMO theoretically yes. We ignored some potential optimization for `#add` and `#read` in past so we're suffering from performance regressions that are beyond reasonable. When we take care of this, things could be better. This PR optimizes `#add` and brings `PKLookup` back to its original speed (locally). I think this it is ready for review. I'll raise another PR to try to optimize `#read`, probably reduce the regression for `Fuzzy` queries. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Optimize outputs accumulating as MSB VLong outputs sharing more output prefix [lucene]
gf2121 commented on code in PR #12661: URL: https://github.com/apache/lucene/pull/12661#discussion_r1363317643 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java: ## @@ -118,13 +118,11 @@ long readVLongOutput(DataInput in) throws IOException { * Package private for testing. */ static long readMSBVLong(DataInput in) throws IOException { -long l = 0L; -while (true) { - byte b = in.readByte(); +byte b = in.readByte(); Review Comment: I keep this change here as i think it is more readable :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org