Re: [PR] Fix jacoco coverage tests (add createClassLoader to replicator permissions) [lucene]
dweiss merged PR #12684: URL: https://github.com/apache/lucene/pull/12684 -- 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_r1360502250 ## lucene/core/src/java/org/apache/lucene/util/hnsw/InitializedHnswGraphBuilder.java: ## @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.util.hnsw; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.util.CollectionUtil; + +/** + * This creates a graph builder that is initialized with the provided HnswGraph. This is useful for + * merging HnswGraphs from multiple segments. + * + * @lucene.internal + */ +public final class InitializedHnswGraphBuilder extends HnswGraphBuilder { Review Comment: @msokolov I think we can just have `int[]`, where `int[oldOrd]=newOrd` since old vector ordinals are continuous. -- 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] Record if block API has been used in SegmentsInfo [lucene]
s1monw commented on code in PR #12685: URL: https://github.com/apache/lucene/pull/12685#discussion_r1360623450 ## lucene/core/src/java/org/apache/lucene/index/IndexWriter.java: ## @@ -3368,9 +3368,15 @@ public void addIndexesReaderMerge(MergePolicy.OneMerge merge) throws IOException String mergedName = newSegmentName(); Directory mergeDirectory = mergeScheduler.wrapForMerge(merge, directory); int numSoftDeleted = 0; +boolean hasBlocks = false; for (MergePolicy.MergeReader reader : merge.getMergeReader()) { CodecReader leaf = reader.codecReader; numDocs += leaf.numDocs(); + if (reader.reader == null) { +hasBlocks = true; // NOCOMMIT: can we just assume that it has blocks and go with worst case here? Review Comment: note there is a NOCOMMIT here that needs to be resolved -- 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] Record if block API has been used in SegmentsInfo [lucene]
s1monw commented on code in PR #12685: URL: https://github.com/apache/lucene/pull/12685#discussion_r1360627901 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99SegmentInfoFormat.java: ## @@ -0,0 +1,236 @@ +/* + * 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 java.io.IOException; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.index.*; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.store.*; +import org.apache.lucene.util.Version; + +/** + * Lucene 9.9 Segment info format. + * + * Files: + * + * + * .si: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, + * Attributes, IndexSort, Footer + * + * + * Data types: + * + * + * Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + * SegSize --> {@link DataOutput#writeInt Int32} + * SegVersion --> {@link DataOutput#writeString String} + * SegMinVersion --> {@link DataOutput#writeString String} + * Files --> {@link DataOutput#writeSetOfStrings Set} + * Diagnostics,Attributes --> {@link DataOutput#writeMapOfStrings Map } + * IsCompoundFile --> {@link DataOutput#writeByte Int8} + * HasBlocks --> {@link DataOutput#writeByte Int8} + * IndexSort --> {@link DataOutput#writeVInt Int32} count, followed by {@code count} + * SortField + * SortField --> {@link DataOutput#writeString String} sort class, followed by a per-sort + * bytestream (see {@link SortFieldProvider#readSortField(DataInput)}) + * Footer --> {@link CodecUtil#writeFooter CodecFooter} + * + * + * Field Descriptions: + * + * + * SegVersion is the code version that created the segment. + * SegMinVersion is the minimum code version that contributed documents to the segment. + * SegSize is the number of documents contained in the segment index. + * IsCompoundFile records whether the segment is written as a compound file or not. If this is + * -1, the segment is not a compound file. If it is 1, the segment is a compound file. + * HasBlocks records whether the segment contains documents written as a block and guarantees + * consecutive document ids for all documents in the block + * The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid, for + * each segment it creates. It includes metadata like the current Lucene version, OS, Java + * version, why the segment was created (merge, flush, addIndexes), etc. + * Files is a list of files referred to by this segment. + * + * + * @see SegmentInfos + * @lucene.experimental + */ +public class Lucene99SegmentInfoFormat extends SegmentInfoFormat { + + /** File extension used to store {@link SegmentInfo}. */ + public static final String SI_EXTENSION = "si"; + + static final String CODEC_NAME = "Lucene90SegmentInfo"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + /** Sole constructor. */ + public Lucene99SegmentInfoFormat() {} + + @Override + public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) + throws IOException { +final String fileName = IndexFileNames.segmentFileName(segment, "", SI_EXTENSION); +try (ChecksumIndexInput input = dir.openChecksumInput(fileName)) { + Throwable priorE = null; + SegmentInfo si = null; + try { +CodecUtil.checkIndexHeader( +input, CODEC_NAME, VERSION_START, VERSION_CURRENT, segmentID, ""); +si = parseSegmentInfo(dir, input, segment, segmentID); + } catch (Throwable exception) { +priorE = exception; + } finally { +CodecUtil.checkFooter(input, priorE); + } + return si; +} + } + + private SegmentInfo parseSegmentInfo( + Directory dir, DataInput input, String segment, byte[] segmentID) throws IOException { +final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt()); +byte hasMinVersion = input.readByte(); +final Version minVer
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
mikemccand commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1360701669 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that). +1 -- I like this approach. Let's not special case the "build FST in memory and immediately use it from memory" case. Let the OS cache the bytes we write and then keep them hot (in RAM) when we immediately open it for reading. That's a nice standalone simplification. > UPDATE: The above seems to be a breaking change and might not be a small, incremental one, as NodeHash needs the FST to be initialized first so that it can search over the previously written nodes. It indeed reads from the still-writing FST, but only to confirm whether a newly created node is already in the `NodeHash`. `NodeHash` is really a `Set`. Normally a Set would hold pointers to java objects as its values, but in our case, instead of holding a true object pointer, we hold a pointer back into the FST since the FST has the node already. We could consider changing that so that `NodeHash` holds a copy of the full node itself (a `byte[]` or a slice into a shared `byte[]`). This is a bit wasteful in RAM, but it'd fully decouple `NodeHash` from the growing / appending FST's `byte[]`? > If we are doing that, then we can get rid of the getReverseBytesReader. However one issue remains: we still need getReverseBytesReaderForSuffixSharing for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode. Aha! It sounds like Tantivy indeed does store-by-value in its LRU suffix cache, not store-by-reference like Lucene does today? Yet another option would be to consider allowing a still open `IndexOutput` to be read by an `IndexInput`? This would be a change in the semantics of our IO abstractions (`Directory` and its babies), and likely problematic for some filesystems (HDFS?), but for most common filesystems would be easily supported. Then we wouldn't have to duplicate the nodes in the `NodeHash` and in the FST. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
mikemccand commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1360715823 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: Actually, while it may seem RAM-wasteful to fully copy the `byte[]` node that are stored in `NodeHash`, if it then (eventually?) enables append-only writing of the FST, net/net that will likely be a RAM savings overall, once we finish with the [LRU NodeHash PR](https://github.com/apache/lucene/pull/12633). So I think that PR is a precursor to this one? This already long pole is growing longer! Like Pinocchio's nose:  I'll try to make time to wrap that PR up soon. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Record if block API has been used in SegmentInfo [lucene]
s1monw commented on code in PR #12685: URL: https://github.com/apache/lucene/pull/12685#discussion_r1360837043 ## lucene/core/src/java/org/apache/lucene/index/IndexWriter.java: ## @@ -3368,9 +3368,15 @@ public void addIndexesReaderMerge(MergePolicy.OneMerge merge) throws IOException String mergedName = newSegmentName(); Directory mergeDirectory = mergeScheduler.wrapForMerge(merge, directory); int numSoftDeleted = 0; +boolean hasBlocks = false; for (MergePolicy.MergeReader reader : merge.getMergeReader()) { CodecReader leaf = reader.codecReader; numDocs += leaf.numDocs(); + if (reader.reader == null) { +hasBlocks = true; // NOCOMMIT: can we just assume that it has blocks and go with worst case here? Review Comment: ```JAVA if (reader.reader == null) { CodecReader unwrap = FilterCodecReader.unwrap(leaf); if (SegmentReader.class.isAssignableFrom(unwrap.getClass())) { hasBlocks = ((SegmentReader) unwrap).getSegmentInfo().info.getHasBlocks(); } else { hasBlocks = false; } } else { hasBlocks |= reader.reader.getSegmentInfo().info.getHasBlocks(); } ``` this would be another option for best effort -- 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_r1360866889 ## 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: Unrelated to this PR and just my thought that would it be more maintainable in the long run if we only have a single way to build the FST through the Builder? -- 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_r1360875178 ## 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: So does this means we still need to read from the in-writing FST? I'm wondering would it be possible to only read from the cache, then we can decouple the FST from NodeHash. -- 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 segmentInfos replace doesn't set userData [lucene]
msfroh commented on code in PR #12626: URL: https://github.com/apache/lucene/pull/12626#discussion_r1360878505 ## lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java: ## @@ -1996,6 +1996,41 @@ public void testGetCommitData() throws Exception { dir.close(); } + @Test Review Comment: It was the case that Lucene unit tests preferred to avoid the `@Test` annotation, going instead with the JUnit 3 convention where public methods whose name start with `test...` would indicate test methods. I think the annotation on `testGetCommitData()` above must have slipped through (11 years ago). -- 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 segmentInfos replace doesn't set userData [lucene]
msfroh commented on code in PR #12626: URL: https://github.com/apache/lucene/pull/12626#discussion_r1360880697 ## lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java: ## @@ -1996,6 +1996,41 @@ public void testGetCommitData() throws Exception { dir.close(); } + @Test Review Comment: Looking through the other tests, I think this is an opportunity to clean up that `@Test` annotation from `testGetCommitData()` to make the whole test class consistent (since that's the only other occurrence of `@Test` 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] read MSB VLong in new way [lucene]
gf2121 commented on PR #12661: URL: https://github.com/apache/lucene/pull/12661#issuecomment-1764814636 I made some effort to speed up the `add` operation for `BytesRef`, getting a tiny improvement: > Baseline: after https://github.com/apache/lucene/pull/12631; Candidate: this patch; Luceneutil Random seed = 0 ``` TaskQPS baseline StdDevQPS my_modified_version StdDevPct diff p-value Fuzzy1 50.96 (1.1%) 51.31 (1.6%)0.7% ( -1% -3%) 0.114 Wildcard 58.12 (2.2%) 58.55 (2.1%)0.8% ( -3% -5%) 0.267 Respell 32.20 (1.7%) 32.51 (1.3%)1.0% ( -2% -4%) 0.044 Fuzzy2 36.56 (1.2%) 37.00 (1.2%)1.2% ( -1% -3%) 0.002 PKLookup 106.36 (2.0%) 108.68 (2.4%)2.2% ( -2% -6%) 0.002 ``` But still far more slower than origin patch: > 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 Fuzzy1 53.95 (1.4%) 51.22 (1.3%) -5.1% ( -7% - -2%) 0.000 PKLookup 114.52 (2.5%) 108.73 (2.7%) -5.0% ( -9% -0%) 0.000 Fuzzy2 38.19 (1.2%) 36.89 (1.3%) -3.4% ( -5% -0%) 0.000 Respell 33.41 (2.2%) 32.37 (2.4%) -3.1% ( -7% -1%) 0.000 Wildcard 59.00 (2.6%) 58.15 (2.5%) -1.4% ( -6% -3%) 0.072 ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[I] `IndexOrDocValuesQuery` does not support query highlighting [lucene]
harshavamsi opened a new issue, #12686: URL: https://github.com/apache/lucene/issues/12686 ### Description While working with the `IndexOrDocValuesQuery`, I noticed that highlighting was broken. This is potentially caused by the extract function that does not check if the query is indexordocvalue. https://github.com/apache/lucene/blob/main/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java#L128 ### Version and environment details Lucene version 9.8.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.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] Add timeouts to github jobs. [lucene]
dweiss opened a new pull request, #12687: URL: https://github.com/apache/lucene/pull/12687 Estimates taken from empirical run times (actions history), with a generous buffer 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] 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_r136233 ## lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java: ## @@ -0,0 +1,189 @@ +/* + * 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.lucene95; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +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.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; + +/** + * This selects the biggest Hnsw graph from the provided merge state and initializes a new + * HnswGraphBuilder with that graph as a starting point. + */ +public class IncrementalHnswGraphMerger { Review Comment: ++ This would make sense for a ConcurrentMerger & "BetterFutureHnswGraphMerger" or whatever it will be. -- 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_r1361112199 ## lucene/core/src/java/org/apache/lucene/codecs/lucene95/IncrementalHnswGraphMerger.java: ## @@ -0,0 +1,189 @@ +/* + * 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.lucene95; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +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.hnsw.HnswGraph; +import org.apache.lucene.util.hnsw.HnswGraphBuilder; +import org.apache.lucene.util.hnsw.InitializedHnswGraphBuilder; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; + +/** + * This selects the biggest Hnsw graph from the provided merge state and initializes a new + * HnswGraphBuilder with that graph as a starting point. + */ +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 + */ + 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 +
Re: [PR] Fix SynonymQuery equals implementation [lucene]
mingshl commented on PR #12260: URL: https://github.com/apache/lucene/pull/12260#issuecomment-1765091198 @romseygeek @mkhludnev, this bug was introduced since 9.4 version, can this PR be back-ported to 9.4.2 to fix the 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] Add new int8 scalar quantization to HNSW codec [lucene]
jmazanec15 commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765145453 Hey @benwtrent, sorry for delay, still looking through change. But 4x space improvement with minimal recall loss is awesome. -- 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_r1361186978 ## 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: I realize this is pretty opaque. I will add some comments here about the algebra used and what we are doing with these numbers. Basically, we are bucketing `float` based on some upper/lower quantiles into `byte` values. In doing this, there is additionally required corrective offsets to account for linear magnitude shifts. This is why `cosine` `dot_product` and `max_inner_product` have a score corrective offset. For `euclidean` the algebra cancelled out all the corrective offsets and we just need a global linear one. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
jmazanec15 commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1357440025 ## 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; Review Comment: >= minQuantile? -- 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] Record if block API has been used in SegmentInfo [lucene]
jpountz commented on code in PR #12685: URL: https://github.com/apache/lucene/pull/12685#discussion_r1361188738 ## lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java: ## @@ -153,6 +157,16 @@ public boolean getUseCompoundFile() { return isCompoundFile; } + /** Returns true if this segment contains documents written as blocks. */ Review Comment: Add a link to `addDocuments` and `updateDocuments`? I wonder if this should be a bit more specific, e.g. "as blocks of 2 docs or more" to clarify that calling `addDocuments` with a single document doesn't count. ## lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java: ## @@ -1815,4 +1815,71 @@ public void testAddIndicesWithSoftDeletes() throws IOException { assertEquals(wrappedReader.numDocs(), writer.getDocStats().maxDoc); IOUtils.close(reader, writer, dir3, dir2, dir1); } + + public void testAddIndicesWithBlocks() throws IOException { +boolean addHasBlocks = random().nextBoolean(); +boolean baseHasBlocks = rarely(); Review Comment: All these cases look worth testing every time intead of randomly picking a single combination? ## lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java: ## @@ -153,6 +157,16 @@ public boolean getUseCompoundFile() { return isCompoundFile; } + /** Returns true if this segment contains documents written as blocks. */ Review Comment: Maybe also mention that this started being recorded in 9.9 and that indexes created earlier than that will return `false` regardless? ## lucene/core/src/java/org/apache/lucene/index/IndexWriter.java: ## @@ -3368,9 +3368,15 @@ public void addIndexesReaderMerge(MergePolicy.OneMerge merge) throws IOException String mergedName = newSegmentName(); Directory mergeDirectory = mergeScheduler.wrapForMerge(merge, directory); int numSoftDeleted = 0; +boolean hasBlocks = false; for (MergePolicy.MergeReader reader : merge.getMergeReader()) { CodecReader leaf = reader.codecReader; numDocs += leaf.numDocs(); + if (reader.reader == null) { +hasBlocks = true; // NOCOMMIT: can we just assume that it has blocks and go with worst case here? Review Comment: Maybe we could we expose getHasBlocks in LeafMetaData to be able to get this information from a CodecReader? -- 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-1765186395 If we're specializing the format anyway, I wonder if we could try different layouts. E.g. another option could be to encode the number of supplementary bytes using unary coding (like UTF8), which should help reduce the number of (hardly predictable) branches and maybe make decoding faster? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. 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 OnHeapHnswGraph's data structure [lucene]
zhaih merged PR #12651: URL: https://github.com/apache/lucene/pull/12651 -- 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]
uschindler commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765287960 Hi, why do we need a new Codec? The Lucebe main file format does not change, olly the HNSW format was exchanged. Because like pistingsfornats and dicvaluesformats, the SPI can detect the format of the HNSW index by reading the file. The top level Codec itself does not need to be versionized. You just need to change its defaults when it writes new segments. Basically the Lucene95 codec was also possibly a useless addition because the top level Codec did not change 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] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765316000 @uschindler so I should just add a new format? It would be a new Lucene99 HNSW format, but keep the default Lucene95 HNSW format? Or can we change the default vector format for the top level Lucene95 codec to a new HNSW format? I definitely don't fully understand how backwards compatibility works. -- 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]
jmazanec15 commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1361261168 ## 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]
jimczi commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765330759 > why do we need a new top-level Codec? The Lucene main file format does not change, only the HNSW format was exchanged. Because like ppostingsfornats and docvaluesformats, the SPI can detect the format of the HNSW index by reading the file and uses SPI to lookup the correct format. That's a good point. I think we'd need to increment the VERSION_CURRENT of the Lucene95HnswVectorsFormat to do the right thing when reading the data and we could avoid the new format entirely since it's exactly the same as before (assuming that quantisation is disabled by default). -- 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] Random access term dictionary [lucene]
Tony-X opened a new pull request, #12688: URL: https://github.com/apache/lucene/pull/12688 ### Description Related issue https://github.com/apache/lucene/issues/12513 Opening this PR early to avoid massive diffs in one-shot - [x] Encode (term type, local ord) in FST TODO: - [ ] Implement bit-packing and unpacking for each term type - [ ] Implement the PostingsFormat -- 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]
uschindler commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765355363 > > why do we need a new top-level Codec? The Lucene main file format does not change, only the HNSW format was exchanged. Because like ppostingsfornats and docvaluesformats, the SPI can detect the format of the HNSW index by reading the file and uses SPI to lookup the correct format. > > That's a good point. I think we'd need to increment the VERSION_CURRENT of the Lucene95HnswVectorsFormat to do the right thing when reading the data and we could avoid the new format entirely since it's exactly the same as before (assuming that quantisation is disabled by default). Actually, if the HNSW format has its own SPI name, when reading indexes it should be chosen automatically by KNNVectorsFormat.forName(): https://lucene.apache.org/core/9_0_0/core/org/apache/lucene/codecs/KnnVectorsFormat.html?is-external=true#forName(java.lang.String) In short: when top level codec reads the index and opens the vector format it would read the SPI name header from file and then load the correct code (possibly the actual one or knew from backwards). That's working like that for years for postings and docvalues. -- 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]
uschindler commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765362790 > @uschindler so I should just add a new format? > > It would be a new Lucene99 HNSW format, but keep the default Lucene95 HNSW format? > > Or can we change the default vector format for the top level Lucene95 codec to a new HNSW format? > > I definitely don't fully understand how backwards compatibility works. No you just create a new HNSW format with a new name. The old one is moved to backwards compatibility. The top level Codec Lucene95Codec ist Format compatible and needs no change in it's identify. Only the part that returns the format for writing must be changed. When it reads an index the format is looked up by its name from the metadata. If that does not work there is a bug in the code reading the segment Infos. So basically, only add a new HNSW format (like you could add a new postings format) but leave the codec intact. Just change the class returned for HNSW when writing. The top level codec only needs to be changed when metadata fotmats or non spi formats change. -- 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]
uschindler commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765382016 I just checked the code, the 9.5 top-level codec addition was useless. Just code duplication. We can't revert it anymore, but we should not repeat that. The only required top-level Format for 9.x is Lucene90Codec. It was duplicated for no reason. -- 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] Create a task executor when executor is not provided [lucene]
sohami commented on code in PR #12606: URL: https://github.com/apache/lucene/pull/12606#discussion_r1361333154 ## lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java: ## @@ -420,13 +418,12 @@ public int count(Query query) throws IOException { } /** - * Returns the leaf slices used for concurrent searching, or null if no {@code Executor} was - * passed to the constructor. + * Returns the leaf slices used for concurrent searching * * @lucene.experimental */ public LeafSlice[] getSlices() { -return (executor == null) ? null : leafSlicesSupplier.get(); +return leafSlicesSupplier.get(); Review Comment: @javanna yes that should be fine. Thanks for checking! -- 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] [BROKEN, for reference only] concurrent hnsw [lucene]
zhaih commented on code in PR #12683: URL: https://github.com/apache/lucene/pull/12683#discussion_r1361334258 ## 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: Will this break back-compatibility? (e.g. when reading a Lucene90 index and we're not able to search on it because the off heap graph there did not implement this?) -- 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]
uschindler commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1765386547 The simplest change is: - Remove Lucene99Codec - In Lucene95Codec just change this: `this.defaultKnnVectorsFormat = new Lucene95HnswVectorsFormat();` to the new format. Done. (when index is written it will write the name of the codec into FieldInfos; when reading the Lucene 95 codec will read the name and looks it up to load the format-sepecific format class) -- 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]
zhaih commented on code in PR #12657: URL: https://github.com/apache/lucene/pull/12657#discussion_r1361341835 ## 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: [I] Exception rising while using QueryTimeout [lucene]
msfroh commented on issue #12032: URL: https://github.com/apache/lucene/issues/12032#issuecomment-1765587096 I started to work on making DrillSidewaysScorer work on windows of doc IDs, when I noticed the following comment added in TestDrillSideways as part of https://github.com/apache/lucene/pull/996/files: ``` // DrillSideways requires the entire range of docs to be scored at once, so it doesn't support // timeouts whose implementation scores one window of doc IDs at a time. ``` Another challenge that I noticed is that the recent change to call `finish` on collectors only after they've finished would require some more changes, as `drillDownLeafCollector` and any `drillSidewaysLeafCollectors` would have their `finish` method called after a single window. -- 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 timeouts to github jobs. [lucene]
dweiss merged PR #12687: URL: https://github.com/apache/lucene/pull/12687 -- 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]
mkhludnev commented on PR #12260: URL: https://github.com/apache/lucene/pull/12260#issuecomment-1765741713 Hi, @mingshl I'm able to cherrypick this fix into branch_9_4, but I'm not sure if there'll be release 9.4.2 ever. -- 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