Merge remote-tracking branch 'upstream/main' into feature/scalar-quantized-off-heap-scoring

This commit is contained in:
Benjamin Trent 2024-08-14 13:52:18 -04:00
commit 40182c948d
279 changed files with 22122 additions and 2017 deletions

View File

@ -0,0 +1,78 @@
#!/usr/bin/env python3
# -*- coding: utf-8 -*-
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import os
import re
import subprocess
import sys
import tempfile
import urllib.request
'''
A simple tool to see diffs between main's version of CHANGES.txt entries for
a given release vs the stable branch's version. It's best to keep these 1)
identical and 2) matching what changes were actually backported to be honest
to users and avoid future annoying conflicts on backport.
'''
# e.g. python3 -u diff_lucene_changes.py branch_9_9 main 9.9.0
#
def get_changes_url(branch_name):
if os.path.isdir(branch_name):
url = f'file://{branch_name}/lucene/CHANGES.txt'
else:
url = f'https://raw.githubusercontent.com/apache/lucene/{branch_name}/lucene/CHANGES.txt'
print(f'NOTE: resolving {branch_name} --> {url}')
return url
def extract_release_section(changes_txt, release_name):
return re.search(f'=======+ Lucene {re.escape(release_name)} =======+(.*?)=======+ Lucene .*? =======+$',
changes_txt.decode('utf-8'), re.MULTILINE | re.DOTALL).group(1).encode('utf-8')
def main():
if len(sys.argv) < 3 or len(sys.argv) > 5:
print('\nUsage: python3 -u dev-tools/scripts/diff_lucene_changes.py <branch1-or-local-clone> <branch2-or-local-clone> <release-name> [diff-commandline-extras]\n')
print(' e.g.: python3 -u dev-tools/scripts/diff_lucene_changes.py branch_9_9 /l/trunk 9.9.0 "-w"\n')
sys.exit(1)
branch1 = sys.argv[1]
branch2 = sys.argv[2]
release_name = sys.argv[3]
if len(sys.argv) > 4:
diff_cl_extras = [sys.argv[4]]
else:
diff_cl_extras = []
branch1_changes = extract_release_section(urllib.request.urlopen(get_changes_url(branch1)).read(),
release_name)
branch2_changes = extract_release_section(urllib.request.urlopen(get_changes_url(branch2)).read(),
release_name)
with tempfile.NamedTemporaryFile() as f1, tempfile.NamedTemporaryFile() as f2:
f1.write(branch1_changes)
f2.write(branch2_changes)
command = ['diff'] + diff_cl_extras + [f1.name, f2.name]
# diff returns non-zero exit status when there are diffs, so don't pass check=True
print(subprocess.run(command, check=False, capture_output=True).stdout.decode('utf-8'))
if __name__ == '__main__':
main()

View File

@ -23,7 +23,7 @@ configure(project(":lucene:core")) {
description "Regenerate gen_ForUtil.py"
group "generation"
def genDir = file("src/java/org/apache/lucene/codecs/lucene99")
def genDir = file("src/java/org/apache/lucene/codecs/lucene912")
def genScript = file("${genDir}/gen_ForUtil.py")
def genOutput = file("${genDir}/ForUtil.java")
@ -96,5 +96,30 @@ configure(project(":lucene:backward-codecs")) {
andThenTasks: ["spotlessJava", "spotlessJavaApply"],
mustRunBefore: [ "compileJava" ]
])
task generateForUtil99Internal() {
description "Regenerate gen_ForUtil.py"
group "generation"
def genDir = file("src/java/org/apache/lucene/backward_codecs/lucene99")
def genScript = file("${genDir}/gen_ForUtil.py")
def genOutput = file("${genDir}/ForUtil.java")
inputs.file genScript
outputs.file genOutput
doLast {
quietExec {
workingDir genDir
executable project.externalTool("python3")
args = [ '-B', genScript ]
}
}
}
regenerate.dependsOn wrapWithPersistentChecksums(generateForUtil99Internal, [
andThenTasks: ["spotlessJava", "spotlessJavaApply"],
mustRunBefore: [ "compileJava" ]
])
}

View File

@ -80,10 +80,6 @@ API Changes
* GITHUB#12875: Ensure token position is always increased in PathHierarchyTokenizer and ReversePathHierarchyTokenizer
and resulting tokens do not overlap. (Michael Froh, Lukáš Vlček)
* GITHUB#12624, GITHUB#12831: Allow FSTCompiler to stream to any DataOutput while building, and
make compile() only return the FSTMetadata. For on-heap (default) use case, please use
FST.fromFSTReader(fstMetadata, fstCompiler.getFSTReader()) to create the FST. (Anh Dung Bui)
* GITHUB#13146, GITHUB#13148: Remove ByteBufferIndexInput and only use MemorySegment APIs
for MMapDirectory. (Uwe Schindler)
@ -112,6 +108,11 @@ API Changes
* GITHUB#13410: Removed Scorer#getWeight (Sanjay Dutt, Adrien Grand)
* GITHUB#13499: Remove deprecated TopScoreDocCollector + TopFieldCollector methods (#create, #createSharedManager) (Jakub Slowinski)
* GITHUB#13632: CandidateMatcher public matching functions (Bryan Jacobowitz)
New Features
---------------------
@ -133,6 +134,13 @@ New Features
DocValuesSkipper abstraction. A new flag is added to FieldType.java that configures whether
to create a "skip index" for doc values. (Ignacio Vera)
* GITHUB#13563: Add levels to doc values skip index. (Ignacio Vera)
* GITHUB#13597: Align doc value skipper interval boundaries when an interval contains a constant
value. (Ignacio Vera)
* GITHUB#13604: Add Kmeans clustering on vectors (Mayya Sharipova, Jim Ferenczi, Tom Veasey)
Improvements
---------------------
@ -243,24 +251,65 @@ Other
* GITHUB#13332: Improve MissingDoclet linter to check records correctly. (Uwe Schindler)
* GITHUB#13499: Remove usage of TopScoreDocCollector + TopFieldCollector deprecated methods (#create, #createSharedManager) (Jakub Slowinski)
======================== Lucene 9.12.0 =======================
API Changes
---------------------
* GITHUB#13281: Mark COSINE VectorSimilarityFunction as deprecated. (Pulkit Gupta)
* GITHUB#13469: Expose FlatVectorsFormat as a first-class format; can be configured using a custom Codec. (Michael Sokolov)
* GITHUB#13469: Expose FlatVectorsFormat as a first-class format; can be configured using a custom Codec. (Michael Sokolov)
* GITHUB#13612: Hunspell: add Suggester#proceedPastRep to avoid losing relevant suggestions. (Peter Gromov)
* GITHUB#13603: Introduced `IndexSearcher#searchLeaf(LeafReaderContext, Weight, Collector)` protected method to
facilitate customizing per-leaf behavior of search without requiring to override
`search(LeafReaderContext[], Weight, Collector)` which requires overriding the entire loop across the leaves (Luca Cavanna)
* GITHUB#13559: Add BitSet#nextSetBit(int, int) to get the index of the first set bit in range. (Egor Potemkin)
* GITHUB#13568: Add DoubleValuesSource#toSortableLongDoubleValuesSource and
MultiDoubleValuesSource#toSortableMultiLongValuesSource methods. (Shradha Shankar)
* GITHUB#13568: Add CollectorOwner class that wraps CollectorManager, and handles list of Collectors and results.
Add IndexSearcher#search method that takes CollectorOwner. (Egor Potemkin)
* GITHUB#13568: Add DrillSideways#search method that supports any collector types for any drill-sideways dimensions
or drill-down. (Egor Potemkin)
New Features
---------------------
(No changes)
* GITHUB#13430: Allow configuring the search concurrency via
TieredMergePolicy#setTargetSearchConcurrency. This in-turn instructs the
merge policy to try to have at least this number of segments on the highest
tier. (Adrien Grand, Carlos Delgado)
* GITHUB#13517: Allow configuring the search concurrency on LogDocMergePolicy
and LogByteSizeMergePolicy via a new #setTargetConcurrency setter.
(Adrien Grand)
* GITHUB#13568: Add sandbox facets module to compute facets while collecting. (Egor Potemkin, Shradha Shankar)
Improvements
---------------------
* GITHUB#13548: Refactor and javadoc update for KNN vector writer classes. (Patrick Zhai)
* GITHUB#13562: Add Intervals.regexp and Intervals.range methods to produce IntervalsSource
for regexp and range queries. (Mayya Sharipova)
* GITHUB#13625: Remove BitSet#nextSetBit code duplication. (Greg Miller)
* GITHUB#13285: Early terminate graph searches of AbstractVectorSimilarityQuery to follow timeout set from
IndexSearcher#setTimeout(QueryTimeout). (Kaival Parikh)
* GITHUB#13633: Add ability to read/write knn vector values to a MemoryIndex. (Ben Trent)
* GITHUB#12627: patch HNSW graphs to improve reachability of all nodes from entry points
* GITHUB#13201: Better cost estimation on MultiTermQuery over few terms. (Michael Froh)
Optimizations
---------------------
@ -282,6 +331,30 @@ Optimizations
* GITHUB#13538: Slightly reduce heap usage for HNSW and scalar quantized vector writers. (Ben Trent)
* GITHUB#12100: WordBreakSpellChecker.suggestWordBreaks now does a breadth first search, allowing it to return
better matches with fewer evaluations (hossman)
* GITHUB#13582: Stop requiring MaxScoreBulkScorer's outer window from having at
least INNER_WINDOW_SIZE docs. (Adrien Grand)
* GITHUB#13570, GITHUB#13574, GITHUB#13535: Avoid performance degradation with closing shared Arenas.
Closing many individual index files can potentially lead to a degradation in execution performance.
Index files are mmapped one-to-one with the JDK's foreign shared Arena. The JVM deoptimizes the top
few frames of all threads when closing a shared Arena (see JDK-8335480). We mitigate this situation
by 1) using a confined Arena where appropriate, and 2) grouping files from the same segment to a
single shared Arena. (Chris Hegarty, Michael Gibney, Uwe Schindler)
* GITHUB#13585: Lucene912PostingsFormat, the new default postings format, now
only has 2 levels of skip data, which are inlined into postings instead of
being stored at the end of postings lists. This translates into better
performance for queries that need skipping such as conjunctions.
(Adrien Grand)
* GITHUB#13581: OnHeapHnswGraph no longer allocates a lock for every graph node (Mike Sokolov)
* GITHUB#13636: Optimizations to the decoding logic of blocks of postings.
(Adrien Grand, Uwe Schindler)
Changes in runtime behavior
---------------------
@ -302,6 +375,11 @@ Bug Fixes
* GITHUB#13553: Correct RamUsageEstimate for scalar quantized knn vector formats so that raw vectors are correctly
accounted for. (Ben Trent)
* GITHUB#13615: Correct scalar quantization when used in conjunction with COSINE similarity. Vectors are normalized
before quantization to ensure the cosine similarity is correctly calculated. (Ben Trent)
* GITHUB#13627: Fix race condition on flush for DWPT seqNo generation. (Ben Trent, Ao Li)
Other
--------------------
(No changes)
@ -531,6 +609,10 @@ API Changes
* GITHUB#12854: Mark DrillSideways#createDrillDownFacetsCollector as @Deprecated. (Greg Miller)
* GITHUB#12624, GITHUB#12831: Allow FSTCompiler to stream to any DataOutput while building, and
make compile() only return the FSTMetadata. For on-heap (default) use case, please use
FST.fromFSTReader(fstMetadata, fstCompiler.getFSTReader()) to create the FST. (Anh Dung Bui)
New Features
---------------------
* GITHUB#12679: Add support for similarity-based vector searches using [Byte|Float]VectorSimilarityQuery. Uses a new
@ -538,6 +620,12 @@ New Features
better-scoring nodes are available, or the best candidate is below a score of `traversalSimilarity` in the lowest
level. (Aditya Prakash, Kaival Parikh)
* GITHUB#12829: For indices newly created as of 9.10.0 onwards, IndexWriter preserves document blocks indexed via
IndexWriter#addDocuments or IndexWriter#updateDocuments also when index sorting is configured. Document blocks are
maintained alongside their parent documents during sort and merge. IndexWriterConfig accepts a parent field that is used
to maintain block orders if index sorting is used. Note, this is fully optional in Lucene 9.x while will be mandatory for
indices that use document blocks together with index sorting as of 10.0.0. (Simon Willnauer)
* GITHUB#12336: Index additional data per facet label in the taxonomy. (Shai Erera, Egor Potemkin, Mike McCandless,
Stefan Vodita)
@ -629,7 +717,6 @@ Build
Other
---------------------
* GITHUB#11023: Removing some dead code in CheckIndex. (Jakub Slowinski)
* GITHUB#11023: Removing @lucene.experimental tags in testXXX methods in CheckIndex. (Jakub Slowinski)

View File

@ -31,6 +31,7 @@ class ModifyingSuggester {
private final String misspelled;
private final WordCase wordCase;
private final FragmentChecker fragmentChecker;
private final boolean proceedPastRep;
private final char[] tryChars;
private final Hunspell speller;
@ -39,13 +40,15 @@ class ModifyingSuggester {
LinkedHashSet<Suggestion> result,
String misspelled,
WordCase wordCase,
FragmentChecker checker) {
FragmentChecker checker,
boolean proceedPastRep) {
this.speller = speller;
tryChars = speller.dictionary.tryChars.toCharArray();
this.result = result;
this.misspelled = misspelled;
this.wordCase = wordCase;
fragmentChecker = checker;
this.proceedPastRep = proceedPastRep;
}
/**
@ -125,9 +128,9 @@ class ModifyingSuggester {
boolean hasGoodSuggestions = trySuggestion(word.toUpperCase(Locale.ROOT));
GradedSuggestions repResult = tryRep(word);
if (repResult == GradedSuggestions.Best) return true;
if (repResult == GradedSuggestions.Best && !proceedPastRep) return true;
hasGoodSuggestions |= repResult == GradedSuggestions.Normal;
hasGoodSuggestions |= repResult != GradedSuggestions.None;
if (!speller.dictionary.mapTable.isEmpty()) {
enumerateMapReplacements(word, "", 0);

View File

@ -53,16 +53,21 @@ public class Suggester {
private final Dictionary dictionary;
private final SuggestibleEntryCache suggestibleCache;
private final FragmentChecker fragmentChecker;
private final boolean proceedPastRep;
public Suggester(Dictionary dictionary) {
this(dictionary, null, FragmentChecker.EVERYTHING_POSSIBLE);
this(dictionary, null, FragmentChecker.EVERYTHING_POSSIBLE, false);
}
private Suggester(
Dictionary dictionary, SuggestibleEntryCache suggestibleCache, FragmentChecker checker) {
Dictionary dictionary,
SuggestibleEntryCache suggestibleCache,
FragmentChecker checker,
boolean proceedPastRep) {
this.dictionary = dictionary;
this.suggestibleCache = suggestibleCache;
this.fragmentChecker = checker;
this.proceedPastRep = proceedPastRep;
}
/**
@ -71,8 +76,8 @@ public class Suggester {
* entries are stored as fast-to-iterate plain words instead of highly compressed prefix trees.
*/
public Suggester withSuggestibleEntryCache() {
return new Suggester(
dictionary, SuggestibleEntryCache.buildCache(dictionary.words), fragmentChecker);
SuggestibleEntryCache cache = SuggestibleEntryCache.buildCache(dictionary.words);
return new Suggester(dictionary, cache, fragmentChecker, proceedPastRep);
}
/**
@ -80,7 +85,17 @@ public class Suggester {
* the performance of the "Modification" phase performance.
*/
public Suggester withFragmentChecker(FragmentChecker checker) {
return new Suggester(dictionary, suggestibleCache, checker);
return new Suggester(dictionary, suggestibleCache, checker, proceedPastRep);
}
/**
* Returns a copy of this suggester instance that doesn't stop after encountering acceptable words
* after applying REP rules. By default, Hunspell stops when it finds any, but this behavior may
* not always be desirable, e.g., if we have "REP i ea", "tims" be replaced only by "teams" and
* not "times", which could also be meant.
*/
public Suggester proceedPastRep() {
return new Suggester(dictionary, suggestibleCache, fragmentChecker, true);
}
/**
@ -174,7 +189,8 @@ public class Suggester {
}
boolean hasGoodSuggestions =
new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase, fragmentChecker)
new ModifyingSuggester(
suggestionSpeller, suggestions, word, wordCase, fragmentChecker, proceedPastRep)
.suggest();
if (!hasGoodSuggestions && dictionary.maxNGramSuggestions > 0) {

View File

@ -59,6 +59,14 @@ public class TestSpellChecking extends LuceneTestCase {
public void testRepSuggestions() throws Exception {
doTest("rep");
//noinspection DataFlowIssue
Path aff = Path.of(getClass().getResource("rep.aff").toURI());
Dictionary dictionary = TestAllDictionaries.loadDictionary(aff);
Suggester suggester = new Suggester(dictionary);
assertEquals(List.of("auto's"), suggester.suggestNoTimeout("autos", () -> {}));
assertEquals(
List.of("auto's", "auto"), suggester.proceedPastRep().suggestNoTimeout("autos", () -> {}));
}
public void testPhSuggestions() throws Exception {

View File

@ -0,0 +1,4 @@
{
"lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java": "f31797842f047626df6a1a6b97167bec60269fec",
"lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py": "325f2610974b0e76e278b6445405a098a3763feb"
}

View File

@ -35,6 +35,7 @@ module org.apache.lucene.backward_codecs {
exports org.apache.lucene.backward_codecs.lucene92;
exports org.apache.lucene.backward_codecs.lucene94;
exports org.apache.lucene.backward_codecs.lucene95;
exports org.apache.lucene.backward_codecs.lucene99;
exports org.apache.lucene.backward_codecs.packed;
exports org.apache.lucene.backward_codecs.store;
@ -43,7 +44,8 @@ module org.apache.lucene.backward_codecs {
provides org.apache.lucene.codecs.PostingsFormat with
org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat,
org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat,
org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat;
org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat,
org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat;
provides org.apache.lucene.codecs.KnnVectorsFormat with
org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat,
org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat,
@ -59,5 +61,6 @@ module org.apache.lucene.backward_codecs {
org.apache.lucene.backward_codecs.lucene91.Lucene91Codec,
org.apache.lucene.backward_codecs.lucene92.Lucene92Codec,
org.apache.lucene.backward_codecs.lucene94.Lucene94Codec,
org.apache.lucene.backward_codecs.lucene95.Lucene95Codec;
org.apache.lucene.backward_codecs.lucene95.Lucene95Codec,
org.apache.lucene.backward_codecs.lucene99.Lucene99Codec;
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.store.DataInput;

View File

@ -16,7 +16,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.store.DataInput;

View File

@ -14,12 +14,33 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.util.Objects;
import org.apache.lucene.codecs.*;
import org.apache.lucene.codecs.lucene90.*;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -98,7 +119,7 @@ public class Lucene99Codec extends Codec {
super("Lucene99");
this.storedFieldsFormat =
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultPostingsFormat = new Lucene99PostingsFormat();
this.defaultPostingsFormat = new Lucene912PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat();
this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat();
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState;
@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.IndexOptions;
@ -339,7 +338,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* @lucene.experimental
*/
public final class Lucene99PostingsFormat extends PostingsFormat {
public class Lucene99PostingsFormat extends PostingsFormat {
/**
* Filename extension for document number, frequencies, and skip data. See chapter: <a
@ -374,28 +373,9 @@ public final class Lucene99PostingsFormat extends PostingsFormat {
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
private final int minTermBlockSize;
private final int maxTermBlockSize;
/** Creates {@code Lucene99PostingsFormat} with default settings. */
public Lucene99PostingsFormat() {
this(
Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/**
* Creates {@code Lucene99PostingsFormat} with custom values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
*
* @see
* Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)
*/
public Lucene99PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene99");
Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize;
}
@Override
@ -405,19 +385,7 @@ public final class Lucene99PostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret =
new Lucene90BlockTreeTermsWriter(
state, postingsWriter, minTermBlockSize, maxTermBlockSize);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
throw new UnsupportedOperationException();
}
@Override

View File

@ -14,23 +14,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import static org.apache.lucene.codecs.lucene99.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.POS_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.VERSION_START;
import static org.apache.lucene.backward_codecs.lucene99.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.POS_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.VERSION_START;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Impacts;
import org.apache.lucene.index.ImpactsEnum;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import java.util.AbstractList;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import java.util.Arrays;
@ -61,6 +61,7 @@ public class Lucene99SkipReader extends MultiLevelSkipListReader {
private long lastDocPointer;
private int lastPosBufferUpto;
/** Sole constructor. */
public Lucene99SkipReader(
IndexInput skipStream,
int maxSkipLevels,
@ -98,6 +99,7 @@ public class Lucene99SkipReader extends MultiLevelSkipListReader {
return df % ForUtil.BLOCK_SIZE == 0 ? df - 1 : df;
}
/** Initialize state. */
public void init(
long skipPointer, long docBasePointer, long posBasePointer, long payBasePointer, int df)
throws IOException {
@ -125,22 +127,27 @@ public class Lucene99SkipReader extends MultiLevelSkipListReader {
return lastDocPointer;
}
/** Returns the pointer in the pos file. */
public long getPosPointer() {
return lastPosPointer;
}
/** Return the start offset in the position block. */
public int getPosBufferUpto() {
return lastPosBufferUpto;
}
/** Returns the pointer in the pay file. */
public long getPayPointer() {
return lastPayPointer;
}
/** Return the number of bytes in the pay block that belongs to docs from the previous block. */
public int getPayloadByteUpto() {
return lastPayloadByteUpto;
}
/** Return the next skip doc, no skipping can be performed until this doc. */
public int getNextSkipDoc() {
return skipDoc[0];
}
@ -199,7 +206,7 @@ public class Lucene99SkipReader extends MultiLevelSkipListReader {
return delta;
}
// The default impl skips impacts
/** Read impacts. The default implementation skips them. */
protected void readImpacts(int level, IndexInput skipStream) throws IOException {
skipStream.skipBytes(skipStream.readVInt());
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import java.util.Arrays;
@ -46,10 +46,10 @@ import org.apache.lucene.store.IndexOutput;
* uptos(position, payload). 4. start offset.
*/
public final class Lucene99SkipWriter extends MultiLevelSkipListWriter {
private int[] lastSkipDoc;
private long[] lastSkipDocPointer;
private long[] lastSkipPosPointer;
private long[] lastSkipPayPointer;
private final int[] lastSkipDoc;
private final long[] lastSkipDocPointer;
private final long[] lastSkipPosPointer;
private final long[] lastSkipPayPointer;
private final IndexOutput docOut;
private final IndexOutput posOut;
@ -61,11 +61,12 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter {
private long curPayPointer;
private int curPosBufferUpto;
private int curPayloadByteUpto;
private CompetitiveImpactAccumulator[] curCompetitiveFreqNorms;
private final CompetitiveImpactAccumulator[] curCompetitiveFreqNorms;
private boolean fieldHasPositions;
private boolean fieldHasOffsets;
private boolean fieldHasPayloads;
/** Sole constructor. */
public Lucene99SkipWriter(
int maxSkipLevels,
int blockSize,
@ -84,7 +85,12 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter {
lastSkipPosPointer = new long[maxSkipLevels];
if (payOut != null) {
lastSkipPayPointer = new long[maxSkipLevels];
} else {
lastSkipPayPointer = null;
}
} else {
lastSkipPosPointer = null;
lastSkipPayPointer = null;
}
curCompetitiveFreqNorms = new CompetitiveImpactAccumulator[maxSkipLevels];
for (int i = 0; i < maxSkipLevels; ++i) {
@ -92,6 +98,7 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter {
}
}
/** Reset state for the given index options. */
public void setField(
boolean fieldHasPositions, boolean fieldHasOffsets, boolean fieldHasPayloads) {
this.fieldHasPositions = fieldHasPositions;
@ -211,6 +218,7 @@ public final class Lucene99SkipWriter extends MultiLevelSkipListWriter {
competitiveFreqNorms.clear();
}
/** Write impacts to the given output. */
public static void writeImpacts(CompetitiveImpactAccumulator acc, DataOutput out)
throws IOException {
Collection<Impact> impacts = acc.getCompetitiveFreqNormPairs();

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import java.util.Arrays;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.store.IndexInput;

View File

@ -40,7 +40,7 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.store.DataInput;

View File

@ -0,0 +1,428 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Lucene 9.9 file format.
*
* <h2>Apache Lucene - Index File Formats</h2>
*
* <div>
*
* <ul>
* <li><a href="#Introduction">Introduction</a>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a>
* <li><a href="#Types_of_Fields">Types of Fields</a>
* <li><a href="#Segments">Segments</a>
* <li><a href="#Document_Numbers">Document Numbers</a>
* </ul>
* <li><a href="#Overview">Index Structure Overview</a>
* <li><a href="#File_Naming">File Naming</a>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a>
* <li><a href="#History">History</a>
* <li><a href="#Limitations">Limitations</a>
* </ul>
* </ul>
*
* </div> <a id="Introduction"></a>
*
* <h3>Introduction</h3>
*
* <div>
*
* <p>This document defines the index file formats used in this version of Lucene. If you are using
* a different version of Lucene, please consult the copy of <code>docs/</code> that was distributed
* with the version you are using.
*
* <p>This document attempts to provide a high-level definition of the Apache Lucene file formats.
* </div> <a id="Definitions"></a>
*
* <h3>Definitions</h3>
*
* <div>
*
* <p>The fundamental concepts in Lucene are index, document, field and term.
*
* <p>An index contains a sequence of documents.
*
* <ul>
* <li>A document is a sequence of fields.
* <li>A field is a named sequence of terms.
* <li>A term is a sequence of bytes.
* </ul>
*
* <p>The same sequence of bytes in two different fields is considered a different term. Thus terms
* are represented as a pair: the string naming the field, and the bytes within the field. <a
* id="Inverted_Indexing"></a>
*
* <h4>Inverted Indexing</h4>
*
* <p>Lucene's index stores terms and statistics about those terms in order to make term-based
* search more efficient. Lucene's terms index falls into the family of indexes known as an
* <i>inverted index.</i> This is because it can list, for a term, the documents that contain it.
* This is the inverse of the natural relationship, in which documents list terms. <a
* id="Types_of_Fields"></a>
*
* <h4>Types of Fields</h4>
*
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored in the index
* literally, in a non-inverted manner. Fields that are inverted are called <i>indexed</i>. A field
* may be both stored and indexed.
*
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the text of a field
* may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is
* useful for certain identifier fields to be indexed literally.
*
* <p>See the {@link org.apache.lucene.document.Field Field} java docs for more information on
* Fields. <a id="Segments"></a>
*
* <h4>Segments</h4>
*
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>. Each segment is a
* fully independent index, which could be searched separately. Indexes evolve by:
*
* <ol>
* <li>Creating new segments for newly added documents.
* <li>Merging existing segments.
* </ol>
*
* <p>Searches may involve multiple segments and/or multiple indexes, each index potentially
* composed of a set of segments. <a id="Document_Numbers"></a>
*
* <h4>Document Numbers</h4>
*
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>. The first
* document added to an index is numbered zero, and each subsequent document added gets a number one
* greater than the previous.
*
* <p>Note that a document's number may change, so caution should be taken when storing these
* numbers outside of Lucene. In particular, numbers may change in the following situations:
*
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and must be
* converted before they can be used in a larger context. The standard technique is to
* allocate each segment a range of values, based on the range of numbers used in that
* segment. To convert a document number from a segment to an external value, the segment's
* <i>base</i> document number is added. To convert an external value back to a
* segment-specific value, the segment is identified by the range that the external value is
* in, and the segment's base value is subtracted. For example two five document segments
* might be combined, so that the first segment has a base value of zero, and the second of
* five. Document three from the second segment would have an external value of eight.
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are eventually
* removed as the index evolves through merging. Deleted documents are dropped when segments
* are merged. A freshly-merged segment thus has no gaps in its numbering.
* </ul>
*
* </div> <a id="Overview"></a>
*
* <h3>Index Structure Overview</h3>
*
* <div>
*
* <p>Each segment index maintains the following:
*
* <ul>
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This
* contains metadata about a segment, such as the number of documents, what files it uses, and
* information about how the segment is sorted
* <li>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This
* contains metadata about the set of named fields used in the index.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes are
* field names. These are used to store auxiliary information about the document, such as its
* title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* <li>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term dictionary}.
* A dictionary containing all of the terms used in all of the indexed fields of all of the
* documents. The dictionary also contains the number of documents which contain the term, and
* pointers to the term's frequency and proximity data.
* <li>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Frequency
* data}. For each term in the dictionary, the numbers of all the documents that contain that
* term, and the frequency of the term in that document, unless frequencies are omitted
* ({@link org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
* <li>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Proximity
* data}. For each term in the dictionary, the positions that the term occurs in each
* document. Note that this will not exist if all fields in all documents omit position data.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
* each field in each document, a value is stored that is multiplied into the score for hits
* on that field.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
* field in each document, the term vector (sometimes called document vector) may be stored. A
* term vector consists of term text and term frequency. To add Term Vectors to your index see
* the {@link org.apache.lucene.document.Field Field} constructors
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like
* stored values, these are also keyed by document number, but are generally intended to be
* loaded into main memory for fast access. Whereas stored values are generally intended for
* summary results from searches, per-document values are useful for things like scoring
* factors.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An
* optional file indicating which documents are live.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair
* of files, recording dimensionally indexed fields, to enable fast numeric range filtering
* and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
* intersection (2D, 3D).
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The
* vector format stores numeric vectors in a format optimized for random access and
* computation, supporting high-dimensional nearest-neighbor search.
* </ul>
*
* <p>Details on each of these are provided in their linked pages. </div> <a id="File_Naming"></a>
*
* <h3>File Naming</h3>
*
* <div>
*
* <p>All files belonging to a segment have the same name with varying extensions. The extensions
* correspond to the different file formats described below. When using the Compound File format
* (default for small segments) these files (except for the Segment info file, the Lock file, and
* Deleted documents file) are collapsed into a single .cfs file (see below for details)
*
* <p>Typically, all segments in an index are stored in a single directory, although this is not
* required.
*
* <p>File names are never re-used. That is, when any file is saved to the Directory it is given a
* never before used filename. This is achieved using a simple generations approach. For example,
* the first segments file is segments_1, then segments_2, etc. The generation is a sequential long
* integer represented in alpha-numeric (base 36) form. </div> <a id="file-names"></a>
*
* <h3>Summary of File Extensions</h3>
*
* <div>
*
* <p>The following table summarizes the names and extensions of the files in Lucene:
*
* <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
* <caption>lucene filenames by extension</caption>
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what documents are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}</td>
* <td>.vec, .vem, .veq, vex</td>
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data,
* <code>.vem</code> the vector metadata, <code>.veq</code> the quantized vector data, and <code>.vex</code> the
* hnsw graph data.</td>
* </tr>
* </table>
*
* </div> <a id="Lock_File"></a>
*
* <h3>Lock File</h3>
*
* The write lock, which is stored in the index directory by default, is named "write.lock". If the
* lock directory is different from the index directory then the write lock will be named
* "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index
* directory. When this file is present, a writer is currently modifying the index (adding or
* removing documents). This lock file ensures that only one writer is modifying the index at a
* time. <a id="History"></a>
*
* <h3>History</h3>
*
* <p>Compatibility notes are provided in this document, describing how file formats have changed
* from prior versions:
*
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit
* lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching
* or adding/deleting of docs. When the new segments file is saved (committed), it will be
* written in the new file format (meaning no specific "upgrade" process is needed). But note
* that once a commit has occurred, pre-2.1 Lucene will not be able to read the index.
* <li>In version 2.3, the file format was changed to allow segments to share a single set of doc
* store (vectors &amp; stored fields) files. This allows for faster indexing in certain
* cases. The change is fully backwards compatible (in the same way as the lock-less commits
* change in 2.1).
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified
* UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">LUCENE-510</a> for
* details.
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData may be passed to
* IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N
* file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">LUCENE-1382</a> for
* details. Also, diagnostics were added to each segment written recording details about why
* it was written (due to flush, merge; which OS/JRE was used; etc.). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.
* <li>In version 3.0, compressed fields are no longer written to the index (they can still be
* read, but on merge the new segment will write them, uncompressed). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> for details.
* <li>In version 3.1, segments records the code version that created them. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> for details.
* <li>In version 3.2, numeric fields are written as natively to stored fields file, previously
* they were stored in text format only.
* <li>In version 3.4, fields can omit position data while still indexing term frequencies.
* <li>In version 4.0, the format of the inverted index became extensible via the {@link
* org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues})
* was introduced. Normalization factors need no longer be a single byte, they can be any
* {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be
* unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into
* the postings lists. Payloads can be stored in the term vectors.
* <li>In version 4.1, the format of the postings list changed to use either of FOR compression or
* variable-byte encoding, depending upon the frequency of the term. Terms appearing only once
* were changed to inline directly into the term dictionary. Stored fields are compressed by
* default.
* <li>In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued
* type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields.
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.
* <li>In version 4.8, checksum footers were added to the end of each index file for improved data
* integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32
* checksum of the file.
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is
* suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk: addresses
* for binary fields and ord indexes for multi-valued fields.
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
* <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index
* sorting.
* <li>In version 7.0, DocValues have been improved to better support sparse doc values thanks to
* an iterator API.
* <li>In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term
* freq, normalization factor) pairs that may trigger the maximum score of the block. This
* information is recorded alongside skip data in order to be able to skip blocks of doc ids
* if they may not produce high enough scores. Additionally doc values and norms has been
* extended with jump-tables to make access O(1) instead of O(n), where n is the number of
* elements to skip when advancing in the data.
* <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
* performant encoding that is vectorized.
* <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to allow
* user-defined sorts to be used
* <li>In version 8.7, stored fields compression became adaptive to better handle documents with
* smaller stored fields.
* <li>In version 9.0, vector-valued fields were added.
* <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
* <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
* IndexDISI. ordToDoc mappings was added to .vem.
* <li>In version 9.5, HNSW graph connections were changed to be delta-encoded with vints.
* Additionally, metadata file size improvements were made by delta-encoding nodes by graph
* layer and not writing the node ids for the zeroth layer.
* <li>In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector
* format to utilize int8 quantized vectors for float32 vector search.
* </ul>
*
* <a id="Limitations"></a>
*
* <h3>Limitations</h3>
*
* <div>
*
* <p>Lucene uses a Java <code>int</code> to refer to document numbers, and the index file format
* uses an <code>Int32</code> on-disk to store document numbers. This is a limitation of both the
* index file format and the current implementation. Eventually these should be replaced with either
* <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
* VInt} values which have no limit. </div>
*/
package org.apache.lucene.backward_codecs.lucene99;

View File

@ -22,3 +22,4 @@ org.apache.lucene.backward_codecs.lucene91.Lucene91Codec
org.apache.lucene.backward_codecs.lucene92.Lucene92Codec
org.apache.lucene.backward_codecs.lucene94.Lucene94Codec
org.apache.lucene.backward_codecs.lucene95.Lucene95Codec
org.apache.lucene.backward_codecs.lucene99.Lucene99Codec

View File

@ -16,3 +16,4 @@
org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat
org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat
org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat
org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat

View File

@ -23,12 +23,11 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.backward_codecs.lucene90.Lucene90ScoreSkipReader.MutableImpactList;
import org.apache.lucene.backward_codecs.lucene99.Lucene99SkipWriter;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.lucene90.blocktree.FieldReader;
import org.apache.lucene.codecs.lucene90.blocktree.Stats;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99SkipWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader;
@ -77,22 +76,6 @@ public class TestLucene90PostingsFormat extends BasePostingsFormatTestCase {
d.close();
}
private void shouldFail(int minItemsInBlock, int maxItemsInBlock) {
expectThrows(
IllegalArgumentException.class,
() -> {
new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock);
});
}
public void testInvalidBlockSizes() throws Exception {
shouldFail(0, 0);
shouldFail(10, 8);
shouldFail(-1, 10);
shouldFail(10, -1);
shouldFail(10, 12);
}
public void testImpactSerialization() throws IOException {
// omit norms and omit freqs
doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L)));

View File

@ -693,9 +693,9 @@ public final class Lucene94HnswVectorsWriter extends KnnVectorsWriter {
lastDocID = docID;
}
OnHeapHnswGraph getGraph() {
OnHeapHnswGraph getGraph() throws IOException {
if (vectors.size() > 0) {
return hnswGraphBuilder.getGraph();
return hnswGraphBuilder.getCompletedGraph();
} else {
return null;
}

View File

@ -732,9 +732,9 @@ public final class Lucene95HnswVectorsWriter extends KnnVectorsWriter {
lastDocID = docID;
}
OnHeapHnswGraph getGraph() {
OnHeapHnswGraph getGraph() throws IOException {
if (vectors.size() > 0) {
return hnswGraphBuilder.getGraph();
return hnswGraphBuilder.getCompletedGraph();
} else {
return null;
}

View File

@ -14,22 +14,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import static org.apache.lucene.codecs.lucene99.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.POS_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT;
import static org.apache.lucene.backward_codecs.lucene99.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.POS_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT;
import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.PushPostingsWriterBase;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
public class Lucene99RWPostingsFormat extends Lucene99PostingsFormat {
private final int minTermBlockSize;
private final int maxTermBlockSize;
/** Creates {@code Lucene99PostingsFormat} with default settings. */
public Lucene99RWPostingsFormat() {
this(
Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/**
* Creates {@code Lucene99PostingsFormat} with custom values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
*
* @see
* Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)
*/
public Lucene99RWPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super();
Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret =
new Lucene90BlockTreeTermsWriter(
state, postingsWriter, minTermBlockSize, maxTermBlockSize);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;

View File

@ -19,7 +19,6 @@ package org.apache.lucene.backward_codecs.lucene99;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase;
public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase {

View File

@ -14,22 +14,26 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import static org.apache.lucene.codecs.lucene99.Lucene99ScoreSkipReader.readImpacts;
import static org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.readImpacts;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.MutableImpactList;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.lucene90.blocktree.FieldReader;
import org.apache.lucene.codecs.lucene90.blocktree.Stats;
import org.apache.lucene.codecs.lucene99.Lucene99ScoreSkipReader.MutableImpactList;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.*;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Impact;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@ -41,7 +45,7 @@ import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99PostingsFormat());
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99RWPostingsFormat());
@Override
protected Codec getCodec() {
@ -77,7 +81,7 @@ public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase {
expectThrows(
IllegalArgumentException.class,
() -> {
new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock);
new Lucene99RWPostingsFormat(minItemsInBlock, maxItemsInBlock);
});
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene99;
package org.apache.lucene.backward_codecs.lucene99;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;

View File

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene99;
import java.io.IOException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.tests.util.LuceneTestCase;
public class TestPostingsUtil extends LuceneTestCase {
// checks for bug described in https://github.com/apache/lucene/issues/13373
public void testIntegerOverflow() throws IOException {
final int size = random().nextInt(1, ForUtil.BLOCK_SIZE);
final long[] docDeltaBuffer = new long[size];
final long[] freqBuffer = new long[size];
final int delta = 1 << 30;
docDeltaBuffer[0] = delta;
try (Directory dir = newDirectory()) {
try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) {
// In old implementation, this would cause integer overflow exception.
PostingsUtil.writeVIntBlock(out, docDeltaBuffer, freqBuffer, size, true);
}
long[] restoredDocs = new long[size];
long[] restoredFreqs = new long[size];
try (IndexInput in = dir.openInput("test", IOContext.DEFAULT)) {
PostingsUtil.readVIntBlock(in, restoredDocs, restoredFreqs, size, true, true);
}
assertEquals(delta, restoredDocs[0]);
}
}
}

View File

@ -20,9 +20,9 @@ import static org.apache.lucene.backward_index.TestBasicBackwardsCompatibility.a
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene99.Lucene99Codec;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.document.Document;

View File

@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.benchmark.jmh;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Random;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.codecs.lucene912.ForUtil;
import org.apache.lucene.codecs.lucene912.PostingIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.util.IOUtils;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
@BenchmarkMode(Mode.Throughput)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
@State(Scope.Benchmark)
@Warmup(iterations = 5, time = 1)
@Measurement(iterations = 5, time = 1)
@Fork(
value = 3,
jvmArgsAppend = {"-Xmx1g", "-Xms1g", "-XX:+AlwaysPreTouch"})
public class PostingIndexInputBenchmark {
private Path path;
private Directory dir;
private IndexInput in;
private PostingIndexInput postingIn;
private final ForUtil forUtil = new ForUtil();
private final long[] values = new long[128];
@Param({"5", "6", "7", "8", "9", "10"})
public int bpv;
@Setup(Level.Trial)
public void setup() throws Exception {
path = Files.createTempDirectory("forUtil");
dir = MMapDirectory.open(path);
try (IndexOutput out = dir.createOutput("docs", IOContext.DEFAULT)) {
Random r = new Random(0);
// Write enough random data to not reach EOF while decoding
for (int i = 0; i < 100; ++i) {
out.writeLong(r.nextLong());
}
}
in = dir.openInput("docs", IOContext.DEFAULT);
postingIn = new PostingIndexInput(in, forUtil);
}
@TearDown(Level.Trial)
public void tearDown() throws Exception {
if (dir != null) {
dir.deleteFile("docs");
}
IOUtils.close(in, dir);
in = null;
dir = null;
Files.deleteIfExists(path);
}
@Benchmark
public void decode(Blackhole bh) throws IOException {
in.seek(3); // random unaligned offset
postingIn.decode(bpv, values);
bh.consume(values);
}
@Benchmark
public void decodeAndPrefixSum(Blackhole bh) throws IOException {
in.seek(3); // random unaligned offset
postingIn.decodeAndPrefixSum(bpv, 100, values);
bh.consume(values);
}
}

View File

@ -20,7 +20,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.feeds.QueryMaker;
import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.search.TopScoreDocCollectorManager;
/** Does search w/ a custom collector */
public class SearchWithCollectorTask extends SearchTask {
@ -49,7 +49,8 @@ public class SearchWithCollectorTask extends SearchTask {
protected Collector createCollector() throws Exception {
Collector collector = null;
if (clnName.equalsIgnoreCase("topScoreDoc") == true) {
collector = TopScoreDocCollector.create(numHits(), Integer.MAX_VALUE);
collector =
new TopScoreDocCollectorManager(numHits(), null, Integer.MAX_VALUE, false).newCollector();
} else if (clnName.length() > 0) {
collector = Class.forName(clnName).asSubclass(Collector.class).getConstructor().newInstance();

View File

@ -23,13 +23,13 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene99PostingsWriter}. */
/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene912PostingsWriter}. */
public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
private final int minTermBlockSize;
@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state);
boolean success = false;
try {
@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene99PostingsReader(state);
PostingsReaderBase postingsReader = new Lucene912PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state);

View File

@ -24,7 +24,7 @@ import java.util.TreeMap;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.TransitionAccessor;
// - or: longer dense skip lists than just next byte?
/**
* Wraps {@link Lucene99PostingsFormat} format for on-disk storage, but then at read time loads and
* Wraps {@link Lucene912PostingsFormat} format for on-disk storage, but then at read time loads and
* stores all terms and postings directly in RAM as byte[], int[].
*
* <p><b>WARNING</b>: This is exceptionally RAM intensive: it makes no effort to compress the
@ -97,12 +97,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return PostingsFormat.forName("Lucene99").fieldsConsumer(state);
return PostingsFormat.forName("Lucene912").fieldsConsumer(state);
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
FieldsProducer postings = PostingsFormat.forName("Lucene99").fieldsProducer(state);
FieldsProducer postings = PostingsFormat.forName("Lucene912").fieldsProducer(state);
if (state.context.context() != IOContext.Context.MERGE) {
FieldsProducer loadedPostings;
try {

View File

@ -22,8 +22,8 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
@ -41,7 +41,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state);
boolean success = false;
try {
@ -57,7 +57,7 @@ public final class FSTPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene99PostingsReader(state);
PostingsReaderBase postingsReader = new Lucene912PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new FSTTermsReader(state, postingsReader);

View File

@ -17,13 +17,13 @@
package org.apache.lucene.codecs.uniformsplit;
import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.TermState;
@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator;
/**
* {@link TermState} serializer which encodes each file pointer as a delta relative to a base file
* pointer. It differs from {@link Lucene99PostingsWriter#encodeTerm} which encodes each file
* pointer. It differs from {@link Lucene912PostingsWriter#encodeTerm} which encodes each file
* pointer as a delta relative to the previous file pointer.
*
* <p>It automatically sets the base file pointer to the first valid file pointer for doc start FP,
@ -95,7 +95,7 @@ public class DeltaBaseTermStateSerializer implements Accountable {
/**
* Writes a {@link BlockTermState} to the provided {@link DataOutput}.
*
* <p>Simpler variant of {@link Lucene99PostingsWriter#encodeTerm(DataOutput, FieldInfo,
* <p>Simpler variant of {@link Lucene912PostingsWriter#encodeTerm(DataOutput, FieldInfo,
* BlockTermState, boolean)}.
*/
public void writeTermState(
@ -140,15 +140,12 @@ public class DeltaBaseTermStateSerializer implements Accountable {
termStatesOutput.writeVLong(intTermState.lastPosBlockOffset);
}
}
if (intTermState.skipOffset != -1) {
termStatesOutput.writeVLong(intTermState.skipOffset);
}
}
/**
* Reads a {@link BlockTermState} from the provided {@link DataInput}.
*
* <p>Simpler variant of {@link Lucene99PostingsReader#decodeTerm(DataInput, FieldInfo,
* <p>Simpler variant of {@link Lucene912PostingsReader#decodeTerm(DataInput, FieldInfo,
* BlockTermState, boolean)}.
*
* @param reuse {@link BlockTermState} to reuse; or null to create a new one.
@ -190,9 +187,6 @@ public class DeltaBaseTermStateSerializer implements Accountable {
intTermState.lastPosBlockOffset = termStatesInput.readVLong();
}
}
if (intTermState.docFreq > BLOCK_SIZE) {
intTermState.skipOffset = termStatesInput.readVLong();
}
return intTermState;
}
@ -210,7 +204,6 @@ public class DeltaBaseTermStateSerializer implements Accountable {
termState.docStartFP = 0;
termState.posStartFP = 0;
termState.payStartFP = 0;
termState.skipOffset = -1;
termState.lastPosBlockOffset = -1;
termState.singletonDocID = -1;

View File

@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
@ -113,7 +113,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer termsWriter =
@ -130,7 +130,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene99PostingsReader(state);
PostingsReaderBase postingsReader = new Lucene912PostingsReader(state);
boolean success = false;
try {
FieldsProducer termsReader =

View File

@ -28,7 +28,7 @@
* org.apache.lucene.search.PhraseQuery})
* <li>Quite efficient for {@link org.apache.lucene.search.PrefixQuery}
* <li>Not efficient for spell-check and {@link org.apache.lucene.search.FuzzyQuery}, in this case
* prefer {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat}
* prefer {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat}
* </ul>
*/
package org.apache.lucene.codecs.uniformsplit;

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.KnnByteVectorField;
@ -42,7 +42,7 @@ import org.apache.lucene.tests.index.BaseIndexFileFormatTestCase;
public class TestHnswBitVectorsFormat extends BaseIndexFileFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene99Codec() {
return new Lucene912Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new HnswBitVectorsFormat();

View File

@ -17,7 +17,7 @@
package org.apache.lucene.codecs.lucene90.tests;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState;
/** Test utility class to create mock {@link IntBlockTermState}. */
public class MockTermStateFactory {

View File

@ -1,4 +1,4 @@
{
"lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java": "1292ad354d255b1272ffd3db684aa2ddb2bc49ec",
"lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py": "ab7b63a1b73986cc04e43de1c8f474b97aef5116"
"lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "cc7d40997e2d6500b79c19ff47461ed6e89d2268",
"lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "ba029f2e374e66c6cf315b2c93f4efa6944dfbb8"
}

View File

@ -15,7 +15,7 @@
* limitations under the License.
*/
import org.apache.lucene.codecs.lucene99.Lucene99Codec;
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
/** Lucene Core. */
@SuppressWarnings("module") // the test framework is compiled after the core...
@ -33,6 +33,7 @@ module org.apache.lucene.core {
exports org.apache.lucene.codecs.lucene94;
exports org.apache.lucene.codecs.lucene95;
exports org.apache.lucene.codecs.lucene99;
exports org.apache.lucene.codecs.lucene912;
exports org.apache.lucene.codecs.perfield;
exports org.apache.lucene.codecs;
exports org.apache.lucene.document;
@ -71,7 +72,7 @@ module org.apache.lucene.core {
provides org.apache.lucene.analysis.TokenizerFactory with
org.apache.lucene.analysis.standard.StandardTokenizerFactory;
provides org.apache.lucene.codecs.Codec with
Lucene99Codec;
Lucene912Codec;
provides org.apache.lucene.codecs.DocValuesFormat with
org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
provides org.apache.lucene.codecs.KnnVectorsFormat with
@ -79,7 +80,7 @@ module org.apache.lucene.core {
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat,
org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat;
provides org.apache.lucene.codecs.PostingsFormat with
org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
provides org.apache.lucene.index.SortFieldProvider with
org.apache.lucene.search.SortField.Provider,
org.apache.lucene.search.SortedNumericSortField.Provider,

View File

@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
return LOADER;
}
static Codec defaultCodec = LOADER.lookup("Lucene99");
static Codec defaultCodec = LOADER.lookup("Lucene912");
}
private final String name;

View File

@ -18,8 +18,6 @@ package org.apache.lucene.codecs;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
@ -106,7 +104,7 @@ public final class CompetitiveImpactAccumulator {
}
/** Get the set of competitive freq and norm pairs, ordered by increasing freq and norm. */
public Collection<Impact> getCompetitiveFreqNormPairs() {
public List<Impact> getCompetitiveFreqNormPairs() {
List<Impact> impacts = new ArrayList<>();
int maxFreqForLowerNorms = 0;
for (int i = 0; i < maxFreqs.length; ++i) {
@ -126,7 +124,7 @@ public final class CompetitiveImpactAccumulator {
for (Impact impact : impacts) {
add(impact, freqNormPairs);
}
return Collections.unmodifiableSet(freqNormPairs);
return List.copyOf(freqNormPairs);
}
private void add(Impact newEntry, TreeSet<Impact> freqNormPairs) {

View File

@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.function.BiFunction;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.DocsWithFieldSet;
@ -35,6 +36,7 @@ import org.apache.lucene.internal.hppc.IntIntHashMap;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.VectorScorer;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOFunction;
/** Writes vectors to an index. */
public abstract class KnnVectorsWriter implements Accountable, Closeable {
@ -111,11 +113,11 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable {
}
/** Tracks state of one sub-reader that we are merging */
private static class VectorValuesSub extends DocIDMerger.Sub {
private static class FloatVectorValuesSub extends DocIDMerger.Sub {
final FloatVectorValues values;
VectorValuesSub(MergeState.DocMap docMap, FloatVectorValues values) {
FloatVectorValuesSub(MergeState.DocMap docMap, FloatVectorValues values) {
super(docMap);
this.values = values;
assert values.docID() == -1;
@ -201,61 +203,81 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable {
public static final class MergedVectorValues {
private MergedVectorValues() {}
/** Returns a merged view over all the segment's {@link FloatVectorValues}. */
public static FloatVectorValues mergeFloatVectorValues(
FieldInfo fieldInfo, MergeState mergeState) throws IOException {
private static void validateFieldEncoding(FieldInfo fieldInfo, VectorEncoding expected) {
assert fieldInfo != null && fieldInfo.hasVectorValues();
if (fieldInfo.getVectorEncoding() != VectorEncoding.FLOAT32) {
VectorEncoding fieldEncoding = fieldInfo.getVectorEncoding();
if (fieldEncoding != expected) {
throw new UnsupportedOperationException(
"Cannot merge vectors encoded as [" + fieldInfo.getVectorEncoding() + "] as FLOAT32");
"Cannot merge vectors encoded as [" + fieldEncoding + "] as " + expected);
}
List<VectorValuesSub> subs = new ArrayList<>();
for (int i = 0; i < mergeState.knnVectorsReaders.length; i++) {
KnnVectorsReader knnVectorsReader = mergeState.knnVectorsReaders[i];
}
private static <V, S> List<S> mergeVectorValues(
KnnVectorsReader[] knnVectorsReaders,
MergeState.DocMap[] docMaps,
IOFunction<KnnVectorsReader, V> valuesSupplier,
BiFunction<MergeState.DocMap, V, S> newSub)
throws IOException {
List<S> subs = new ArrayList<>();
for (int i = 0; i < knnVectorsReaders.length; i++) {
KnnVectorsReader knnVectorsReader = knnVectorsReaders[i];
if (knnVectorsReader != null) {
FloatVectorValues values = knnVectorsReader.getFloatVectorValues(fieldInfo.name);
V values = valuesSupplier.apply(knnVectorsReader);
if (values != null) {
subs.add(new VectorValuesSub(mergeState.docMaps[i], values));
subs.add(newSub.apply(docMaps[i], values));
}
}
}
return new MergedFloat32VectorValues(subs, mergeState);
return subs;
}
/** Returns a merged view over all the segment's {@link FloatVectorValues}. */
public static FloatVectorValues mergeFloatVectorValues(
FieldInfo fieldInfo, MergeState mergeState) throws IOException {
validateFieldEncoding(fieldInfo, VectorEncoding.FLOAT32);
return new MergedFloat32VectorValues(
mergeVectorValues(
mergeState.knnVectorsReaders,
mergeState.docMaps,
knnVectorsReader -> {
return knnVectorsReader.getFloatVectorValues(fieldInfo.name);
},
(docMap, values) -> {
return new FloatVectorValuesSub(docMap, values);
}),
mergeState);
}
/** Returns a merged view over all the segment's {@link ByteVectorValues}. */
public static ByteVectorValues mergeByteVectorValues(FieldInfo fieldInfo, MergeState mergeState)
throws IOException {
assert fieldInfo != null && fieldInfo.hasVectorValues();
if (fieldInfo.getVectorEncoding() != VectorEncoding.BYTE) {
throw new UnsupportedOperationException(
"Cannot merge vectors encoded as [" + fieldInfo.getVectorEncoding() + "] as BYTE");
}
List<ByteVectorValuesSub> subs = new ArrayList<>();
for (int i = 0; i < mergeState.knnVectorsReaders.length; i++) {
KnnVectorsReader knnVectorsReader = mergeState.knnVectorsReaders[i];
if (knnVectorsReader != null) {
ByteVectorValues values = knnVectorsReader.getByteVectorValues(fieldInfo.name);
if (values != null) {
subs.add(new ByteVectorValuesSub(mergeState.docMaps[i], values));
}
}
}
return new MergedByteVectorValues(subs, mergeState);
validateFieldEncoding(fieldInfo, VectorEncoding.BYTE);
return new MergedByteVectorValues(
mergeVectorValues(
mergeState.knnVectorsReaders,
mergeState.docMaps,
knnVectorsReader -> {
return knnVectorsReader.getByteVectorValues(fieldInfo.name);
},
(docMap, values) -> {
return new ByteVectorValuesSub(docMap, values);
}),
mergeState);
}
static class MergedFloat32VectorValues extends FloatVectorValues {
private final List<VectorValuesSub> subs;
private final DocIDMerger<VectorValuesSub> docIdMerger;
private final List<FloatVectorValuesSub> subs;
private final DocIDMerger<FloatVectorValuesSub> docIdMerger;
private final int size;
private int docId;
VectorValuesSub current;
FloatVectorValuesSub current;
private MergedFloat32VectorValues(List<VectorValuesSub> subs, MergeState mergeState)
private MergedFloat32VectorValues(List<FloatVectorValuesSub> subs, MergeState mergeState)
throws IOException {
this.subs = subs;
docIdMerger = DocIDMerger.of(subs, mergeState.needsIndexSort);
int totalSize = 0;
for (VectorValuesSub sub : subs) {
for (FloatVectorValuesSub sub : subs) {
totalSize += sub.values.size();
}
size = totalSize;

View File

@ -19,9 +19,13 @@ package org.apache.lucene.codecs.lucene90;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_LEVEL_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_MAX_LEVEL;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
@ -43,7 +47,6 @@ import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
@ -202,70 +205,150 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
docCount = 0;
}
boolean isDone(int skipIndexIntervalSize, int valueCount, long nextValue, int nextDoc) {
if (docCount < skipIndexIntervalSize) {
return false;
}
// Once we reach the interval size, we will keep accepting documents if
// - next doc value is not a multi-value
// - current accumulator only contains a single value and next value is the same value
// - the accumulator is dense and the next doc keeps the density (no gaps)
return valueCount > 1
|| minValue != maxValue
|| minValue != nextValue
|| docCount != nextDoc - minDocID;
}
void accumulate(long value) {
minValue = Math.min(minValue, value);
maxValue = Math.max(maxValue, value);
}
void accumulate(SkipAccumulator other) {
assert minDocID <= other.minDocID && maxDocID < other.maxDocID;
maxDocID = other.maxDocID;
minValue = Math.min(minValue, other.minValue);
maxValue = Math.max(maxValue, other.maxValue);
docCount += other.docCount;
}
void nextDoc(int docID) {
maxDocID = docID;
++docCount;
}
void writeTo(DataOutput output) throws IOException {
output.writeInt(maxDocID);
output.writeInt(minDocID);
output.writeLong(maxValue);
output.writeLong(minValue);
output.writeInt(docCount);
public static SkipAccumulator merge(List<SkipAccumulator> list, int index, int length) {
SkipAccumulator acc = new SkipAccumulator(list.get(index).minDocID);
for (int i = 0; i < length; i++) {
acc.accumulate(list.get(index + i));
}
return acc;
}
}
private void writeSkipIndex(FieldInfo field, DocValuesProducer valuesProducer)
throws IOException {
assert field.hasDocValuesSkipIndex();
// TODO: This disk compression once we introduce levels
long start = data.getFilePointer();
SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
final long start = data.getFilePointer();
final SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
long globalMaxValue = Long.MIN_VALUE;
long globalMinValue = Long.MAX_VALUE;
int globalDocCount = 0;
int maxDocId = -1;
final List<SkipAccumulator> accumulators = new ArrayList<>();
SkipAccumulator accumulator = null;
int counter = 0;
final int maxAccumulators = 1 << (SKIP_INDEX_LEVEL_SHIFT * (SKIP_INDEX_MAX_LEVEL - 1));
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
if (counter == 0) {
accumulator = new SkipAccumulator(doc);
}
accumulator.nextDoc(doc);
for (int i = 0, end = values.docValueCount(); i < end; ++i) {
accumulator.accumulate(values.nextValue());
}
if (++counter == skipIndexIntervalSize) {
final long firstValue = values.nextValue();
if (accumulator != null
&& accumulator.isDone(skipIndexIntervalSize, values.docValueCount(), firstValue, doc)) {
globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
globalMinValue = Math.min(globalMinValue, accumulator.minValue);
globalDocCount += accumulator.docCount;
maxDocId = accumulator.maxDocID;
accumulator.writeTo(data);
counter = 0;
accumulator = null;
if (accumulators.size() == maxAccumulators) {
writeLevels(accumulators);
accumulators.clear();
}
}
if (accumulator == null) {
accumulator = new SkipAccumulator(doc);
accumulators.add(accumulator);
}
accumulator.nextDoc(doc);
accumulator.accumulate(firstValue);
for (int i = 1, end = values.docValueCount(); i < end; ++i) {
accumulator.accumulate(values.nextValue());
}
}
if (counter > 0) {
if (accumulators.isEmpty() == false) {
globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
globalMinValue = Math.min(globalMinValue, accumulator.minValue);
globalDocCount += accumulator.docCount;
maxDocId = accumulator.maxDocID;
accumulator.writeTo(data);
writeLevels(accumulators);
}
meta.writeLong(start); // record the start in meta
meta.writeLong(data.getFilePointer() - start); // record the length
assert globalDocCount == 0 || globalMaxValue >= globalMinValue;
meta.writeLong(globalMaxValue);
meta.writeLong(globalMinValue);
assert globalDocCount <= maxDocId + 1;
meta.writeInt(globalDocCount);
meta.writeInt(maxDocId);
}
private void writeLevels(List<SkipAccumulator> accumulators) throws IOException {
final List<List<SkipAccumulator>> accumulatorsLevels = new ArrayList<>(SKIP_INDEX_MAX_LEVEL);
accumulatorsLevels.add(accumulators);
for (int i = 0; i < SKIP_INDEX_MAX_LEVEL - 1; i++) {
accumulatorsLevels.add(buildLevel(accumulatorsLevels.get(i)));
}
int totalAccumulators = accumulators.size();
for (int index = 0; index < totalAccumulators; index++) {
// compute how many levels we need to write for the current accumulator
final int levels = getLevels(index, totalAccumulators);
// write the number of levels
data.writeByte((byte) levels);
// write intervals in reverse order. This is done so we don't
// need to read all of them in case of slipping
for (int level = levels - 1; level >= 0; level--) {
final SkipAccumulator accumulator =
accumulatorsLevels.get(level).get(index >> (SKIP_INDEX_LEVEL_SHIFT * level));
data.writeInt(accumulator.maxDocID);
data.writeInt(accumulator.minDocID);
data.writeLong(accumulator.maxValue);
data.writeLong(accumulator.minValue);
data.writeInt(accumulator.docCount);
}
}
}
private static List<SkipAccumulator> buildLevel(List<SkipAccumulator> accumulators) {
final int levelSize = 1 << SKIP_INDEX_LEVEL_SHIFT;
final List<SkipAccumulator> collector = new ArrayList<>();
for (int i = 0; i < accumulators.size() - levelSize + 1; i += levelSize) {
collector.add(SkipAccumulator.merge(accumulators, i, levelSize));
}
return collector;
}
private static int getLevels(int index, int size) {
if (Integer.numberOfTrailingZeros(index) >= SKIP_INDEX_LEVEL_SHIFT) {
// TODO: can we do it in constant time rather than linearly with SKIP_INDEX_MAX_LEVEL?
final int left = size - index;
for (int level = SKIP_INDEX_MAX_LEVEL - 1; level > 0; level--) {
final int numberIntervals = 1 << (SKIP_INDEX_LEVEL_SHIFT * level);
if (left >= numberIntervals && index % numberIntervals == 0) {
return level + 1;
}
}
}
return 1;
}
private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer, boolean ords)
throws IOException {
SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);

View File

@ -194,5 +194,36 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat {
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
// number of documents in an interval
private static final int DEFAULT_SKIP_INDEX_INTERVAL_SIZE = 4096;
// bytes on an interval:
// * 1 byte : number of levels
// * 16 bytes: min / max value,
// * 8 bytes: min / max docID
// * 4 bytes: number of documents
private static final long SKIP_INDEX_INTERVAL_BYTES = 29L;
// number of intervals represented as a shift to create a new level, this is 1 << 3 == 8
// intervals.
static final int SKIP_INDEX_LEVEL_SHIFT = 3;
// max number of levels
// Increasing this number, it increases how much heap we need at index time.
// we currently need (1 * 8 * 8 * 8) = 512 accumulators on heap
static final int SKIP_INDEX_MAX_LEVEL = 4;
// number of bytes to skip when skipping a level. It does not take into account the
// current interval that is being read.
static final long[] SKIP_INDEX_JUMP_LENGTH_PER_LEVEL = new long[SKIP_INDEX_MAX_LEVEL];
static {
// Size of the interval minus read bytes (1 byte for level and 4 bytes for maxDocID)
SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[0] = SKIP_INDEX_INTERVAL_BYTES - 5L;
for (int level = 1; level < SKIP_INDEX_MAX_LEVEL; level++) {
// jump from previous level
SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] = SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level - 1];
// nodes added by new level
SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] +=
(1 << (level * SKIP_INDEX_LEVEL_SHIFT)) * SKIP_INDEX_INTERVAL_BYTES;
// remove the byte levels added in the previous level
SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level] -= (1 << ((level - 1) * SKIP_INDEX_LEVEL_SHIFT));
}
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.codecs.lucene90;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_JUMP_LENGTH_PER_LEVEL;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_MAX_LEVEL;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
import java.io.IOException;
@ -1792,28 +1794,55 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
if (input.length() > 0) {
input.prefetch(0, 1);
}
// TODO: should we write to disk the actual max level for this segment?
return new DocValuesSkipper() {
int minDocID = -1;
int maxDocID = -1;
long minValue, maxValue;
int docCount;
final int[] minDocID = new int[SKIP_INDEX_MAX_LEVEL];
final int[] maxDocID = new int[SKIP_INDEX_MAX_LEVEL];
{
for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
minDocID[i] = maxDocID[i] = -1;
}
}
final long[] minValue = new long[SKIP_INDEX_MAX_LEVEL];
final long[] maxValue = new long[SKIP_INDEX_MAX_LEVEL];
final int[] docCount = new int[SKIP_INDEX_MAX_LEVEL];
int levels = 1;
@Override
public void advance(int target) throws IOException {
if (target > entry.maxDocId) {
minDocID = DocIdSetIterator.NO_MORE_DOCS;
maxDocID = DocIdSetIterator.NO_MORE_DOCS;
// skipper is exhausted
for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
minDocID[i] = maxDocID[i] = DocIdSetIterator.NO_MORE_DOCS;
}
} else {
// find next interval
assert target > maxDocID[0] : "target must be bigger that current interval";
while (true) {
maxDocID = input.readInt();
if (maxDocID >= target) {
minDocID = input.readInt();
maxValue = input.readLong();
minValue = input.readLong();
docCount = input.readInt();
levels = input.readByte();
assert levels <= SKIP_INDEX_MAX_LEVEL && levels > 0
: "level out of range [" + levels + "]";
boolean valid = true;
// check if current interval is competitive or we can jump to the next position
for (int level = levels - 1; level >= 0; level--) {
if ((maxDocID[level] = input.readInt()) < target) {
input.skipBytes(SKIP_INDEX_JUMP_LENGTH_PER_LEVEL[level]); // the jump for the level
valid = false;
break;
}
minDocID[level] = input.readInt();
maxValue[level] = input.readLong();
minValue[level] = input.readLong();
docCount[level] = input.readInt();
}
if (valid) {
// adjust levels
while (levels < SKIP_INDEX_MAX_LEVEL && maxDocID[levels] >= target) {
levels++;
}
break;
} else {
input.skipBytes(24);
}
}
}
@ -1821,32 +1850,32 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
@Override
public int numLevels() {
return 1;
return levels;
}
@Override
public int minDocID(int level) {
return minDocID;
return minDocID[level];
}
@Override
public int maxDocID(int level) {
return maxDocID;
return maxDocID[level];
}
@Override
public long minValue(int level) {
return minValue;
return minValue[level];
}
@Override
public long maxValue(int level) {
return maxValue;
return maxValue[level];
}
@Override
public int docCount(int level) {
return docCount;
return docCount[level];
}
@Override

View File

@ -49,9 +49,9 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter;
*
* <pre class="prettyprint">
* // the default: for high performance
* indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_SPEED));
* indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_SPEED));
* // instead for higher performance (but slower):
* // indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_COMPRESSION));
* // indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_COMPRESSION));
* </pre>
*
* <p><b>File formats</b>

View File

@ -78,7 +78,6 @@ public final class FieldReader extends Terms {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.rootCode = rootCode;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// if (DEBUG) {
@ -100,6 +99,14 @@ public final class FieldReader extends Terms {
w.close();
}
*/
BytesRef emptyOutput = metadata.getEmptyOutput();
if (rootCode.equals(emptyOutput) == false) {
// TODO: this branch is never taken
assert false;
this.rootCode = rootCode;
} else {
this.rootCode = emptyOutput;
}
}
long readVLongOutput(DataInput in) throws IOException {

View File

@ -200,6 +200,11 @@ public final class Lucene90BlockTreeTermsReader extends FieldsProducer {
final int docCount = metaIn.readVInt();
BytesRef minTerm = readBytesRef(metaIn);
BytesRef maxTerm = readBytesRef(metaIn);
if (numTerms == 1) {
assert maxTerm.equals(minTerm);
// save heap for edge case of a single term only so min == max
maxTerm = minTerm;
}
if (docCount < 0
|| docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
throw new CorruptIndexException(
@ -270,9 +275,8 @@ public final class Lucene90BlockTreeTermsReader extends FieldsProducer {
throw new CorruptIndexException("invalid bytes length: " + numBytes, in);
}
BytesRef bytes = new BytesRef();
BytesRef bytes = new BytesRef(numBytes);
bytes.length = numBytes;
bytes.bytes = new byte[numBytes];
in.readBytes(bytes.bytes, 0, numBytes);
return bytes;

View File

@ -598,8 +598,6 @@ public final class Lucene90BlockTreeTermsWriter extends FieldsConsumer {
private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
static final BytesRef EMPTY_BYTES_REF = new BytesRef();
private static class StatsWriter {
private final DataOutput out;

View File

@ -277,10 +277,10 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// System.out.println(" skip rewind!");
// }
}
assert length == f.prefix;
assert length == f.prefixLength;
} else {
f.nextEnt = -1;
f.prefix = length;
f.prefixLength = length;
f.state.termBlockOrd = 0;
f.fpOrig = f.fp = fp;
f.lastSubFP = -1;
@ -488,7 +488,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// toHex(targetLabel));
// }
validIndexPrefix = currentFrame.prefix;
validIndexPrefix = currentFrame.prefixLength;
// validIndexPrefix = targetUpto;
currentFrame.scanToFloorFrame(target);
@ -550,7 +550,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
}
// validIndexPrefix = targetUpto;
validIndexPrefix = currentFrame.prefix;
validIndexPrefix = currentFrame.prefixLength;
currentFrame.scanToFloorFrame(target);
@ -772,7 +772,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// targetLabel);
// }
validIndexPrefix = currentFrame.prefix;
validIndexPrefix = currentFrame.prefixLength;
// validIndexPrefix = targetUpto;
currentFrame.scanToFloorFrame(target);
@ -830,7 +830,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
}
// validIndexPrefix = targetUpto;
validIndexPrefix = currentFrame.prefix;
validIndexPrefix = currentFrame.prefixLength;
currentFrame.scanToFloorFrame(target);
@ -868,7 +868,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
while (true) {
SegmentTermsEnumFrame f = getFrame(ord);
assert f != null;
final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefix);
final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefixLength);
if (f.nextEnt == -1) {
out.println(
" frame "
@ -879,7 +879,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
+ f.fp
+ (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "")
+ " prefixLen="
+ f.prefix
+ f.prefixLength
+ " prefix="
+ prefix
+ (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")"))
@ -907,7 +907,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
+ f.fp
+ (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "")
+ " prefixLen="
+ f.prefix
+ f.prefixLength
+ " prefix="
+ prefix
+ " nextEnt="
@ -932,12 +932,14 @@ final class SegmentTermsEnum extends BaseTermsEnum {
}
if (fr.index != null) {
assert !isSeekFrame || f.arc != null : "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix - 1) & 0xFF)) {
if (f.prefixLength > 0
&& isSeekFrame
&& f.arc.label() != (term.byteAt(f.prefixLength - 1) & 0xFF)) {
out.println(
" broken seek state: arc.label="
+ (char) f.arc.label()
+ " vs term byte="
+ (char) (term.byteAt(f.prefix - 1) & 0xFF));
+ (char) (term.byteAt(f.prefixLength - 1) & 0xFF));
throw new RuntimeException("seek state is broken");
}
BytesRef output = Util.get(fr.index, prefix);
@ -965,7 +967,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
if (f == currentFrame) {
break;
}
if (f.prefix == validIndexPrefix) {
if (f.prefixLength == validIndexPrefix) {
isSeekFrame = false;
}
ord++;
@ -1046,7 +1048,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// Note that the seek state (last seek) has been
// invalidated beyond this depth
validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix);
validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefixLength);
// if (DEBUG) {
// System.out.println(" reset validIndexPrefix=" + validIndexPrefix);
// }

View File

@ -59,7 +59,7 @@ final class SegmentTermsEnumFrame {
final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
// Length of prefix shared by all terms in this block
int prefix;
int prefixLength;
// Number of entries (term or sub-block) in this block
int entCount;
@ -318,11 +318,11 @@ final class SegmentTermsEnumFrame {
assert nextEnt != -1 && nextEnt < entCount
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
suffix = suffixLengthsReader.readVInt();
suffixLength = suffixLengthsReader.readVInt();
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.setLength(prefixLength + suffixLength);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
suffixesReader.readBytes(ste.term.bytes(), prefixLength, suffixLength);
ste.termExists = true;
}
@ -346,11 +346,11 @@ final class SegmentTermsEnumFrame {
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffix = code >>> 1;
suffixLength = code >>> 1;
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.setLength(prefixLength + suffixLength);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
suffixesReader.readBytes(ste.term.bytes(), prefixLength, suffixLength);
if ((code & 1) == 0) {
// A normal term
ste.termExists = true;
@ -375,7 +375,7 @@ final class SegmentTermsEnumFrame {
// floor blocks we "typically" get
public void scanToFloorFrame(BytesRef target) {
if (!isFloor || target.length <= prefix) {
if (!isFloor || target.length <= prefixLength) {
// if (DEBUG) {
// System.out.println(" scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" +
// target.length + " vs prefix=" + prefix);
@ -383,7 +383,7 @@ final class SegmentTermsEnumFrame {
return;
}
final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
final int targetLabel = target.bytes[target.offset + prefixLength] & 0xFF;
// if (DEBUG) {
// System.out.println(" scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" +
@ -497,7 +497,7 @@ final class SegmentTermsEnumFrame {
// Used only by assert
private boolean prefixMatches(BytesRef target) {
for (int bytePos = 0; bytePos < prefix; bytePos++) {
for (int bytePos = 0; bytePos < prefixLength; bytePos++) {
if (target.bytes[target.offset + bytePos] != ste.term.byteAt(bytePos)) {
return false;
}
@ -553,7 +553,7 @@ final class SegmentTermsEnumFrame {
}
private int startBytePos;
private int suffix;
private int suffixLength;
private long subCode;
CompressionAlgorithm compressionAlg = CompressionAlgorithm.NO_COMPRESSION;
@ -584,7 +584,7 @@ final class SegmentTermsEnumFrame {
do {
nextEnt++;
suffix = suffixLengthsReader.readVInt();
suffixLength = suffixLengthsReader.readVInt();
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
@ -596,16 +596,16 @@ final class SegmentTermsEnumFrame {
// }
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
suffixesReader.skipBytes(suffixLength);
// Loop over bytes in the suffix, comparing to the target
final int cmp =
Arrays.compareUnsigned(
suffixBytes,
startBytePos,
startBytePos + suffix,
startBytePos + suffixLength,
target.bytes,
target.offset + prefix,
target.offset + prefixLength,
target.offset + target.length);
if (cmp < 0) {
@ -674,7 +674,7 @@ final class SegmentTermsEnumFrame {
assert prefixMatches(target);
suffix = suffixLengthsReader.readVInt();
suffixLength = suffixLengthsReader.readVInt();
// TODO early terminate when target length unequals suffix + prefix.
// But we need to keep the same status with scanToTermLeaf.
int start = nextEnt;
@ -684,16 +684,16 @@ final class SegmentTermsEnumFrame {
while (start <= end) {
int mid = (start + end) >>> 1;
nextEnt = mid + 1;
startBytePos = mid * suffix;
startBytePos = mid * suffixLength;
// Binary search bytes in the suffix, comparing to the target.
cmp =
Arrays.compareUnsigned(
suffixBytes,
startBytePos,
startBytePos + suffix,
startBytePos + suffixLength,
target.bytes,
target.offset + prefix,
target.offset + prefixLength,
target.offset + target.length);
if (cmp < 0) {
start = mid + 1;
@ -701,7 +701,7 @@ final class SegmentTermsEnumFrame {
end = mid - 1;
} else {
// Exact match!
suffixesReader.setPosition(startBytePos + suffix);
suffixesReader.setPosition(startBytePos + suffixLength);
fillTerm();
// if (DEBUG) System.out.println(" found!");
return SeekStatus.FOUND;
@ -724,14 +724,14 @@ final class SegmentTermsEnumFrame {
// If binary search ended at the less term, and greater term exists.
// We need to advance to the greater term.
if (cmp < 0) {
startBytePos += suffix;
startBytePos += suffixLength;
nextEnt++;
}
suffixesReader.setPosition(startBytePos + suffix);
suffixesReader.setPosition(startBytePos + suffixLength);
fillTerm();
} else {
seekStatus = SeekStatus.END;
suffixesReader.setPosition(startBytePos + suffix);
suffixesReader.setPosition(startBytePos + suffixLength);
if (exactOnly) {
fillTerm();
}
@ -769,7 +769,7 @@ final class SegmentTermsEnumFrame {
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffix = code >>> 1;
suffixLength = code >>> 1;
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
@ -782,7 +782,7 @@ final class SegmentTermsEnumFrame {
// }
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
suffixesReader.skipBytes(suffixLength);
ste.termExists = (code & 1) == 0;
if (ste.termExists) {
state.termBlockOrd++;
@ -796,9 +796,9 @@ final class SegmentTermsEnumFrame {
Arrays.compareUnsigned(
suffixBytes,
startBytePos,
startBytePos + suffix,
startBytePos + suffixLength,
target.bytes,
target.offset + prefix,
target.offset + prefixLength,
target.offset + target.length);
if (cmp < 0) {
@ -819,7 +819,8 @@ final class SegmentTermsEnumFrame {
// us to position to the next term after
// the target, so we must recurse into the
// sub-frame(s):
ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, prefix + suffix);
ste.currentFrame =
ste.pushFrame(null, ste.currentFrame.lastSubFP, prefixLength + suffixLength);
ste.currentFrame.loadBlock();
while (ste.currentFrame.next()) {
ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
@ -864,9 +865,9 @@ final class SegmentTermsEnumFrame {
}
private void fillTerm() {
final int termLength = prefix + suffix;
final int termLength = prefixLength + suffixLength;
ste.term.setLength(termLength);
ste.term.grow(termLength);
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix);
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefixLength, suffixLength);
}
}

View File

@ -116,10 +116,10 @@ public class Stats {
nonFloorBlockCount++;
}
if (blockCountByPrefixLen.length <= frame.prefix) {
blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1 + frame.prefix);
if (blockCountByPrefixLen.length <= frame.prefixLength) {
blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1 + frame.prefixLength);
}
blockCountByPrefixLen[frame.prefix]++;
blockCountByPrefixLen[frame.prefixLength]++;
startBlockCount++;
totalBlockSuffixBytes += frame.totalSuffixBytes;
totalUncompressedBlockSuffixBytes += frame.suffixesReader.length();

View File

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.PackedInts;
/** Utility class to encode/decode increasing sequences of 128 integers. */
final class ForDeltaUtil {
// IDENTITY_PLUS_ONE[i] == i+1
private static final long[] IDENTITY_PLUS_ONE = new long[ForUtil.BLOCK_SIZE];
static {
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
IDENTITY_PLUS_ONE[i] = i + 1;
}
}
private static void prefixSumOfOnes(long[] arr, long base) {
System.arraycopy(IDENTITY_PLUS_ONE, 0, arr, 0, ForUtil.BLOCK_SIZE);
// This loop gets auto-vectorized
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
arr[i] += base;
}
}
private final ForUtil forUtil;
ForDeltaUtil(ForUtil forUtil) {
this.forUtil = forUtil;
}
/**
* Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code
* longs} are expected to be deltas between consecutive values.
*/
void encodeDeltas(long[] longs, DataOutput out) throws IOException {
if (longs[0] == 1 && PForUtil.allEqual(longs)) { // happens with very dense postings
out.writeByte((byte) 0);
} else {
long or = 0;
for (long l : longs) {
or |= l;
}
assert or != 0;
final int bitsPerValue = PackedInts.bitsRequired(or);
out.writeByte((byte) bitsPerValue);
forUtil.encode(longs, bitsPerValue, out);
}
}
/** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */
void decodeAndPrefixSum(PostingIndexInput in, long base, long[] longs) throws IOException {
final int bitsPerValue = Byte.toUnsignedInt(in.in.readByte());
if (bitsPerValue == 0) {
prefixSumOfOnes(longs, base);
} else {
in.decodeAndPrefixSum(bitsPerValue, base, longs);
}
}
void skip(DataInput in) throws IOException {
final int bitsPerValue = Byte.toUnsignedInt(in.readByte());
in.skipBytes(forUtil.numBytes(bitsPerValue));
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,217 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat;
import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 9.12 index format
*
* <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene99 package documentation for file format details.
* @lucene.experimental
*/
public class Lucene912Codec extends Codec {
/** Configuration option for the codec. */
public enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED),
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION);
private final Lucene90StoredFieldsFormat.Mode storedMode;
private Mode(Lucene90StoredFieldsFormat.Mode storedMode) {
this.storedMode = Objects.requireNonNull(storedMode);
}
}
private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene94FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene99SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene90CompoundFormat();
private final NormsFormat normsFormat = new Lucene90NormsFormat();
private final PostingsFormat defaultPostingsFormat;
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene912Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat defaultDVFormat;
private final DocValuesFormat docValuesFormat =
new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene912Codec.this.getDocValuesFormatForField(field);
}
};
private final KnnVectorsFormat defaultKnnVectorsFormat;
private final KnnVectorsFormat knnVectorsFormat =
new PerFieldKnnVectorsFormat() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return Lucene912Codec.this.getKnnVectorsFormatForField(field);
}
};
private final StoredFieldsFormat storedFieldsFormat;
/** Instantiates a new codec. */
public Lucene912Codec() {
this(Mode.BEST_SPEED);
}
/**
* Instantiates a new codec, specifying the stored fields compression mode to use.
*
* @param mode stored fields compression mode to use for newly flushed/merged segments.
*/
public Lucene912Codec(Mode mode) {
super("Lucene912");
this.storedFieldsFormat =
new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultPostingsFormat = new Lucene912PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat();
this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat();
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public final CompoundFormat compoundFormat() {
return compoundFormat;
}
@Override
public final PointsFormat pointsFormat() {
return new Lucene90PointsFormat();
}
@Override
public final KnnVectorsFormat knnVectorsFormat() {
return knnVectorsFormat;
}
/**
* Returns the postings format that should be used for writing new segments of <code>field</code>.
*
* <p>The default implementation always returns "Lucene912".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation,
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPostingsFormat;
}
/**
* Returns the docvalues format that should be used for writing new segments of <code>field</code>
* .
*
* <p>The default implementation always returns "Lucene90".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation.
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
/**
* Returns the vectors format that should be used for writing new segments of <code>field</code>
*
* <p>The default implementation always returns "Lucene99HnswVectorsFormat".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation.
*/
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return defaultKnnVectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,492 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 9.12 postings format, which encodes postings in packed integer blocks for fast decode.
*
* <p>Basic idea:
*
* <ul>
* <li><b>Packed Blocks and VInt Blocks</b>:
* <p>In packed blocks, integers are encoded with the same bit width ({@link PackedInts packed
* format}): the block size (i.e. number of integers inside block) is fixed (currently 128).
* Additionally blocks that are all the same value are encoded in an optimized way.
* <p>In VInt blocks, integers are encoded as {@link DataOutput#writeVInt VInt}: the block
* size is variable.
* <li><b>Block structure</b>:
* <p>When the postings are long enough, Lucene912PostingsFormat will try to encode most
* integer data as a packed block.
* <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as
* two packed blocks, while the remaining 3 are encoded as one VInt block.
* <p>Different kinds of data are always encoded separately into different packed blocks, but
* may possibly be interleaved into the same VInt block.
* <p>This strategy is applied to pairs: &lt;document number, frequency&gt;, &lt;position,
* payload length&gt;, &lt;position, offset start, offset length&gt;, and &lt;position,
* payload length, offsetstart, offset length&gt;.
* <li><b>Skipdata</b>:
* <p>Skipdata is interleaved with blocks on 2 levels. Level 0 skip data is interleaved
* between every packed block. Level 1 skip data is interleaved between every 32 packed
* blocks.
* <li><b>Positions, Payloads, and Offsets</b>:
* <p>A position is an integer indicating where the term occurs within one document. A payload
* is a blob of metadata associated with current position. An offset is a pair of integers
* indicating the tokenized start/end offsets for given term in current position: it is
* essentially a specialized payload.
* <p>When payloads and offsets are not omitted, numPositions==numPayloads==numOffsets
* (assuming a null payload contributes one count). As mentioned in block structure, it is
* possible to encode these three either combined or separately.
* <p>In all cases, payloads and offsets are stored together. When encoded as a packed block,
* position data is separated out as .pos, while payloads and offsets are encoded in .pay
* (payload metadata will also be stored directly in .pay). When encoded as VInt blocks, all
* these three are stored interleaved into the .pos (so is payload metadata).
* <p>With this strategy, the majority of payload and offset data will be outside .pos file.
* So for queries that require only position data, running on a full index with payloads and
* offsets, this reduces disk pre-fetches.
* </ul>
*
* <p>Files and detailed format:
*
* <ul>
* <li><code>.tim</code>: <a href="#Termdictionary">Term Dictionary</a>
* <li><code>.tip</code>: <a href="#Termindex">Term Index</a>
* <li><code>.doc</code>: <a href="#Frequencies">Frequencies and Skip Data</a>
* <li><code>.pos</code>: <a href="#Positions">Positions</a>
* <li><code>.pay</code>: <a href="#Payloads">Payloads and Offsets</a>
* </ul>
*
* <a id="Termdictionary"></a>
*
* <dl>
* <dd><b>Term Dictionary</b>
* <p>The .tim file contains the list of terms in each field along with per-term statistics
* (such as docfreq) and pointers to the frequencies, positions, payload and skip data in the
* .doc, .pos, and .pay files. See {@link Lucene90BlockTreeTermsWriter} for more details on
* the format.
* <p>NOTE: The term dictionary can plug into different postings implementations: the postings
* writer/reader are actually responsible for encoding and decoding the PostingsHeader and
* TermMetadata sections described here:
* <ul>
* <li>PostingsHeader --&gt; Header, PackedBlockSize
* <li>TermMetadata --&gt; (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?,
* PayFPDelta?
* <li>Header, --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>PackedBlockSize, SingletonDocID --&gt; {@link DataOutput#writeVInt VInt}
* <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta --&gt; {@link
* DataOutput#writeVLong VLong}
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <p>Notes:
* <ul>
* <li>Header is a {@link CodecUtil#writeIndexHeader IndexHeader} storing the version
* information for the postings.
* <li>PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width
* is determined by the largest integer. Smaller block size result in smaller variance
* among width of integers hence smaller indexes. Larger block size result in more
* efficient bulk i/o hence better acceleration. This value should always be a multiple
* of 64, currently fixed as 128 as a tradeoff. It is also the skip interval used to
* accelerate {@link org.apache.lucene.index.PostingsEnum#advance(int)}.
* <li>DocFPDelta determines the position of this term's TermFreqs within the .doc file. In
* particular, it is the difference of file offset between this term's data and previous
* term's data (or zero, for the first term in the block).On disk it is stored as the
* difference from previous value in sequence.
* <li>PosFPDelta determines the position of this term's TermPositions within the .pos file.
* While PayFPDelta determines the position of this term's &lt;TermPayloads,
* TermOffsets?&gt; within the .pay file. Similar to DocFPDelta, it is the difference
* between two file positions (or neglected, for fields that omit payloads and offsets).
* <li>PosVIntBlockFPDelta determines the position of this term's last TermPosition in last
* pos packed block within the .pos file. It is synonym for PayVIntBlockFPDelta or
* OffsetVIntBlockFPDelta. This is actually used to indicate whether it is necessary to
* load following payloads and offsets from .pos instead of .pay. Every time a new block
* of positions are to be loaded, the PostingsReader will use this value to check
* whether current block is packed format or VInt. When packed format, payloads and
* offsets are fetched from .pay, otherwise from .pos. (this value is neglected when
* total number of positions i.e. totalTermFreq is less or equal to PackedBlockSize).
* <li>SingletonDocID is an optimization when a term only appears in one document. In this
* case, instead of writing a file pointer to the .doc file (DocFPDelta), and then a
* VIntBlock at that location, the single document ID is written to the term dictionary.
* </ul>
* </dl>
*
* <a id="Termindex"></a>
*
* <dl>
* <dd><b>Term Index</b>
* <p>The .tip file contains an index into the term dictionary, so that it can be accessed
* randomly. See {@link Lucene90BlockTreeTermsWriter} for more details on the format.
* </dl>
*
* <a id="Frequencies"></a>
*
* <dl>
* <dd><b>Frequencies and Skip Data</b>
* <p>The .doc file contains the lists of documents which contain each term, along with the
* frequency of the term in that document (except when frequencies are omitted: {@link
* IndexOptions#DOCS}). Skip data is saved at the end of each term's postings. The skip data
* is saved once for the entire postings list.
* <ul>
* <li>docFile(.doc) --&gt; Header, &lt;TermFreqs&gt;<sup>TermCount</sup>, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>TermFreqs --&gt; &lt;PackedBlock32&gt; <sup>PackedDocBlockNum/32</sup>, VIntBlock?
* <li>PackedBlock32 --&gt; Level1SkipData, &lt;PackedBlock&gt; <sup>32</sup>
* <li>PackedBlock --&gt; Level0SkipData, PackedDocDeltaBlock, PackedFreqBlock?
* <li>VIntBlock --&gt;
* &lt;DocDelta[,Freq?]&gt;<sup>DocFreq-PackedBlockSize*PackedDocBlockNum</sup>
* <li>Level1SkipData --&gt; DocDelta, DocFPDelta, Skip1NumBytes?, ImpactLength?, Impacts?,
* PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto?
* <li>Level0SkipData --&gt; Skip0NumBytes, DocDelta, DocFPDelta, PackedBlockLength,
* ImpactLength?, Impacts?, PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto?
* <li>PackedFreqBlock --&gt; {@link PackedInts PackedInts}, uses patching
* <li>PackedDocDeltaBlock --&gt; {@link PackedInts PackedInts}, does not use patching
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <p>Notes:
* <ul>
* <li>PackedDocDeltaBlock is theoretically generated from two steps:
* <ol>
* <li>Calculate the difference between each document number and previous one, and get
* a d-gaps list (for the first document, use absolute value);
* <li>For those d-gaps from first one to
* PackedDocBlockNum*PackedBlockSize<sup>th</sup>, separately encode as packed
* blocks.
* </ol>
* If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step.
* <li>VIntBlock stores remaining d-gaps (along with frequencies when possible) with a
* format that encodes DocDelta and Freq:
* <p>DocDelta: if frequencies are indexed, this determines both the document number and
* the frequency. In particular, DocDelta/2 is the difference between this document
* number and the previous document number (or zero when this is the first document in a
* TermFreqs). When DocDelta is odd, the frequency is one. When DocDelta is even, the
* frequency is read as another VInt. If frequencies are omitted, DocDelta contains the
* gap (not multiplied by 2) between document numbers and no frequency information is
* stored.
* <p>For example, the TermFreqs for a term which occurs once in document seven and
* three times in document eleven, with frequencies indexed, would be the following
* sequence of VInts:
* <p>15, 8, 3
* <p>If frequencies were omitted ({@link IndexOptions#DOCS}) it would be this sequence
* of VInts instead:
* <p>7,4
* <li>PackedDocBlockNum is the number of packed blocks for current term's docids or
* frequencies. In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize)
* <li>On skip data, DocDelta is the delta between the last doc of the previous block - or
* -1 if there is no previous block - and the last doc of this block. This helps know by
* how much the doc ID should be incremented in case the block gets skipped.
* <li>Skip0Length is the length of skip data at level 0. Encoding it is useful when skip
* data is never needed to quickly skip over skip data, e.g. if only using nextDoc(). It
* is also used when only the first fields of skip data are needed, in order to skip
* over remaining fields without reading them.
* <li>ImpactLength and Impacts are only stored if frequencies are indexed.
* <li>Since positions and payloads are also block encoded, the skip should skip to related
* block first, then fetch the values according to in-block offset. PosFPSkip and
* PayFPSkip record the file offsets of related block in .pos and .pay, respectively.
* While PosBlockOffset indicates which value to fetch inside the related block
* (PayBlockOffset is unnecessary since it is always equal to PosBlockOffset). Same as
* DocFPSkip, the file offsets are relative to the start of current term's TermFreqs,
* and stored as a difference sequence.
* <li>PayByteUpto indicates the start offset of the current payload. It is equivalent to
* the sum of the payload lengths in the current block up to PosBlockOffset
* <li>ImpactLength is the total length of CompetitiveFreqDelta and CompetitiveNormDelta
* pairs. CompetitiveFreqDelta and CompetitiveNormDelta are used to safely skip score
* calculation for uncompetitive documents; See {@link
* org.apache.lucene.codecs.CompetitiveImpactAccumulator} for more details.
* </ul>
* </dl>
*
* <a id="Positions"></a>
*
* <dl>
* <dd><b>Positions</b>
* <p>The .pos file contains the lists of positions that each term occurs at within documents.
* It also sometimes stores part of payloads and offsets for speedup.
* <ul>
* <li>PosFile(.pos) --&gt; Header, &lt;TermPositions&gt; <sup>TermCount</sup>, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>TermPositions --&gt; &lt;PackedPosDeltaBlock&gt; <sup>PackedPosBlockNum</sup>,
* VIntBlock?
* <li>VIntBlock --&gt; &lt;PositionDelta[, PayloadLength?], PayloadData?, OffsetDelta?,
* OffsetLength?&gt;<sup>PosVIntCount</sup>
* <li>PackedPosDeltaBlock --&gt; {@link PackedInts PackedInts}
* <li>PositionDelta, OffsetDelta, OffsetLength --&gt; {@link DataOutput#writeVInt VInt}
* <li>PayloadData --&gt; {@link DataOutput#writeByte byte}<sup>PayLength</sup>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <p>Notes:
* <ul>
* <li>TermPositions are order by term (terms are implicit, from the term dictionary), and
* position values for each term document pair are incremental, and ordered by document
* number.
* <li>PackedPosBlockNum is the number of packed blocks for current term's positions,
* payloads or offsets. In particular, PackedPosBlockNum =
* floor(totalTermFreq/PackedBlockSize)
* <li>PosVIntCount is the number of positions encoded as VInt format. In particular,
* PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize
* <li>The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock
* in chapter <a href="#Frequencies">Frequencies and Skip Data</a>.
* <li>PositionDelta is, if payloads are disabled for the term's field, the difference
* between the position of the current occurrence in the document and the previous
* occurrence (or zero, if this is the first occurrence in this document). If payloads
* are enabled for the term's field, then PositionDelta/2 is the difference between the
* current and the previous position. If payloads are enabled and PositionDelta is odd,
* then PayloadLength is stored, indicating the length of the payload at the current
* term position.
* <li>For example, the TermPositions for a term which occurs as the fourth term in one
* document, and as the fifth and ninth term in a subsequent document, would be the
* following sequence of VInts (payloads disabled):
* <p>4, 5, 4
* <li>PayloadData is metadata associated with the current term position. If PayloadLength
* is stored at the current position, then it indicates the length of this payload. If
* PayloadLength is not stored, then this payload has the same length as the payload at
* the previous position.
* <li>OffsetDelta/2 is the difference between this position's startOffset from the previous
* occurrence (or zero, if this is the first occurrence in this document). If
* OffsetDelta is odd, then the length (endOffset-startOffset) differs from the previous
* occurrence and an OffsetLength follows. Offset data is only written for {@link
* IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.
* </ul>
* </dl>
*
* <a id="Payloads"></a>
*
* <dl>
* <dd><b>Payloads and Offsets</b>
* <p>The .pay file will store payloads and offsets associated with certain term-document
* positions. Some payloads and offsets will be separated out into .pos file, for performance
* reasons.
* <ul>
* <li>PayFile(.pay): --&gt; Header, &lt;TermPayloads?, TermOffsets?&gt;
* <sup>TermCount</sup>, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>TermPayloads --&gt; &lt;PackedPayLengthBlock, SumPayLength, PayData&gt;
* <sup>PackedPayBlockNum</sup>
* <li>TermOffsets --&gt; &lt;PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock&gt;
* <sup>PackedPayBlockNum</sup>
* <li>PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --&gt;
* {@link PackedInts PackedInts}
* <li>SumPayLength --&gt; {@link DataOutput#writeVInt VInt}
* <li>PayData --&gt; {@link DataOutput#writeByte byte}<sup>SumPayLength</sup>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
* </ul>
* <p>Notes:
* <ul>
* <li>The order of TermPayloads/TermOffsets will be the same as TermPositions, note that
* part of payload/offsets are stored in .pos.
* <li>The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is
* the same as PackedFreqBlock in chapter <a href="#Frequencies">Frequencies and Skip
* Data</a>. While PackedStartDeltaBlock follows a same procedure as
* PackedDocDeltaBlock.
* <li>PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also
* synonym for PackedOffsetBlockNum.
* <li>SumPayLength is the total length of payloads written within one block, should be the
* sum of PayLengths in one packed block.
* <li>PayLength in PackedPayLengthBlock is the length of each payload associated with the
* current position.
* </ul>
* </dl>
*
* @lucene.experimental
*/
public final class Lucene912PostingsFormat extends PostingsFormat {
/** Filename extension for some small metadata about how postings are encoded. */
public static final String META_EXTENSION = "psm";
/**
* Filename extension for document number, frequencies, and skip data. See chapter: <a
* href="#Frequencies">Frequencies and Skip Data</a>
*/
public static final String DOC_EXTENSION = "doc";
/** Filename extension for positions. See chapter: <a href="#Positions">Positions</a> */
public static final String POS_EXTENSION = "pos";
/**
* Filename extension for payloads and offsets. See chapter: <a href="#Payloads">Payloads and
* Offsets</a>
*/
public static final String PAY_EXTENSION = "pay";
/** Size of blocks. */
public static final int BLOCK_SIZE = ForUtil.BLOCK_SIZE;
public static final int BLOCK_MASK = BLOCK_SIZE - 1;
/** We insert skip data on every block and every SKIP_FACTOR=32 blocks. */
public static final int LEVEL1_FACTOR = 32;
/** Total number of docs covered by level 1 skip data: 32 * 128 = 4,096 */
public static final int LEVEL1_NUM_DOCS = LEVEL1_FACTOR * BLOCK_SIZE;
public static final int LEVEL1_MASK = LEVEL1_NUM_DOCS - 1;
static final String TERMS_CODEC = "Lucene90PostingsWriterTerms";
static final String META_CODEC = "Lucene912PostingsWriterMeta";
static final String DOC_CODEC = "Lucene912PostingsWriterDoc";
static final String POS_CODEC = "Lucene912PostingsWriterPos";
static final String PAY_CODEC = "Lucene912PostingsWriterPay";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
private final int minTermBlockSize;
private final int maxTermBlockSize;
/** Creates {@code Lucene912PostingsFormat} with default settings. */
public Lucene912PostingsFormat() {
this(
Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/**
* Creates {@code Lucene912PostingsFormat} with custom values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
*
* @see
* Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)
*/
public Lucene912PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene912");
Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret =
new Lucene90BlockTreeTermsWriter(
state, postingsWriter, minTermBlockSize, maxTermBlockSize);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene912PostingsReader(state);
boolean success = false;
try {
FieldsProducer ret = new Lucene90BlockTreeTermsReader(postingsReader, state);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
/**
* Holds all state required for {@link Lucene912PostingsReader} to produce a {@link
* org.apache.lucene.index.PostingsEnum} without re-seeking the terms dict.
*
* @lucene.internal
*/
public static final class IntBlockTermState extends BlockTermState {
/** file pointer to the start of the doc ids enumeration, in {@link #DOC_EXTENSION} file */
public long docStartFP;
/** file pointer to the start of the positions enumeration, in {@link #POS_EXTENSION} file */
public long posStartFP;
/** file pointer to the start of the payloads enumeration, in {@link #PAY_EXTENSION} file */
public long payStartFP;
/**
* file offset for the last position in the last block, if there are more than {@link
* ForUtil#BLOCK_SIZE} positions; otherwise -1
*
* <p>One might think to use total term frequency to track how many positions are left to read
* as we decode the blocks, and decode the last block differently when num_left_positions &lt;
* BLOCK_SIZE. Unfortunately this won't work since the tracking will be messed up when we skip
* blocks as the skipper will only tell us new position offset (start of block) and number of
* positions to skip for that block, without telling us how many positions it has skipped.
*/
public long lastPosBlockOffset;
/**
* docid when there is a single pulsed posting, otherwise -1. freq is always implicitly
* totalTermFreq in this case.
*/
public int singletonDocID;
/** Sole constructor. */
public IntBlockTermState() {
lastPosBlockOffset = -1;
singletonDocID = -1;
}
@Override
public IntBlockTermState clone() {
IntBlockTermState other = new IntBlockTermState();
other.copyFrom(this);
return other;
}
@Override
public void copyFrom(TermState _other) {
super.copyFrom(_other);
IntBlockTermState other = (IntBlockTermState) _other;
docStartFP = other.docStartFP;
posStartFP = other.posStartFP;
payStartFP = other.payStartFP;
lastPosBlockOffset = other.lastPosBlockOffset;
singletonDocID = other.singletonDocID;
}
@Override
public String toString() {
return super.toString()
+ " docStartFP="
+ docStartFP
+ " posStartFP="
+ posStartFP
+ " payStartFP="
+ payStartFP
+ " lastPosBlockOffset="
+ lastPosBlockOffset
+ " singletonDocID="
+ singletonDocID;
}
}
}

View File

@ -0,0 +1,683 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_MASK;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.PushPostingsWriterBase;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Impact;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** Writer for {@link Lucene912PostingsFormat}. */
public class Lucene912PostingsWriter extends PushPostingsWriterBase {
static final IntBlockTermState EMPTY_STATE = new IntBlockTermState();
IndexOutput metaOut;
IndexOutput docOut;
IndexOutput posOut;
IndexOutput payOut;
IntBlockTermState lastState;
// Holds starting file pointers for current term:
private long docStartFP;
private long posStartFP;
private long payStartFP;
final long[] docDeltaBuffer;
final long[] freqBuffer;
private int docBufferUpto;
final long[] posDeltaBuffer;
final long[] payloadLengthBuffer;
final long[] offsetStartDeltaBuffer;
final long[] offsetLengthBuffer;
private int posBufferUpto;
private byte[] payloadBytes;
private int payloadByteUpto;
private int level0LastDocID;
private long level0LastPosFP;
private long level0LastPayFP;
private int level1LastDocID;
private long level1LastPosFP;
private long level1LastPayFP;
private int docID;
private int lastDocID;
private int lastPosition;
private int lastStartOffset;
private int docCount;
private final PForUtil pforUtil;
private final ForDeltaUtil forDeltaUtil;
private boolean fieldHasNorms;
private NumericDocValues norms;
private final CompetitiveImpactAccumulator level0FreqNormAccumulator =
new CompetitiveImpactAccumulator();
private final CompetitiveImpactAccumulator level1CompetitiveFreqNormAccumulator =
new CompetitiveImpactAccumulator();
private int maxNumImpactsAtLevel0;
private int maxImpactNumBytesAtLevel0;
private int maxNumImpactsAtLevel1;
private int maxImpactNumBytesAtLevel1;
/** Scratch output that we use to be able to prepend the encoded length, e.g. impacts. */
private final ByteBuffersDataOutput scratchOutput = ByteBuffersDataOutput.newResettableInstance();
/**
* Output for a single block. This is useful to be able to prepend skip data before each block,
* which can only be computed once the block is encoded. The content is then typically copied to
* {@link #level1Output}.
*/
private final ByteBuffersDataOutput level0Output = ByteBuffersDataOutput.newResettableInstance();
/**
* Output for groups of 32 blocks. This is useful to prepend skip data for these 32 blocks, which
* can only be done once we have encoded these 32 blocks. The content is then typically copied to
* {@link #docCount}.
*/
private final ByteBuffersDataOutput level1Output = ByteBuffersDataOutput.newResettableInstance();
/** Sole constructor. */
public Lucene912PostingsWriter(SegmentWriteState state) throws IOException {
String metaFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.META_EXTENSION);
String docFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.DOC_EXTENSION);
metaOut = state.directory.createOutput(metaFileName, state.context);
IndexOutput posOut = null;
IndexOutput payOut = null;
boolean success = false;
try {
docOut = state.directory.createOutput(docFileName, state.context);
CodecUtil.writeIndexHeader(
metaOut, META_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.writeIndexHeader(
docOut, DOC_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
final ForUtil forUtil = new ForUtil();
forDeltaUtil = new ForDeltaUtil(forUtil);
pforUtil = new PForUtil(forUtil);
if (state.fieldInfos.hasProx()) {
posDeltaBuffer = new long[BLOCK_SIZE];
String posFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.POS_EXTENSION);
posOut = state.directory.createOutput(posFileName, state.context);
CodecUtil.writeIndexHeader(
posOut, POS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
if (state.fieldInfos.hasPayloads()) {
payloadBytes = new byte[128];
payloadLengthBuffer = new long[BLOCK_SIZE];
} else {
payloadBytes = null;
payloadLengthBuffer = null;
}
if (state.fieldInfos.hasOffsets()) {
offsetStartDeltaBuffer = new long[BLOCK_SIZE];
offsetLengthBuffer = new long[BLOCK_SIZE];
} else {
offsetStartDeltaBuffer = null;
offsetLengthBuffer = null;
}
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
String payFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name,
state.segmentSuffix,
Lucene912PostingsFormat.PAY_EXTENSION);
payOut = state.directory.createOutput(payFileName, state.context);
CodecUtil.writeIndexHeader(
payOut, PAY_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
}
} else {
posDeltaBuffer = null;
payloadLengthBuffer = null;
offsetStartDeltaBuffer = null;
offsetLengthBuffer = null;
payloadBytes = null;
}
this.payOut = payOut;
this.posOut = posOut;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut);
}
}
docDeltaBuffer = new long[BLOCK_SIZE];
freqBuffer = new long[BLOCK_SIZE];
}
@Override
public IntBlockTermState newTermState() {
return new IntBlockTermState();
}
@Override
public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException {
CodecUtil.writeIndexHeader(
termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
termsOut.writeVInt(BLOCK_SIZE);
}
@Override
public void setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
lastState = EMPTY_STATE;
fieldHasNorms = fieldInfo.hasNorms();
}
@Override
public void startTerm(NumericDocValues norms) {
docStartFP = docOut.getFilePointer();
if (writePositions) {
posStartFP = posOut.getFilePointer();
level1LastPosFP = level0LastPosFP = posStartFP;
if (writePayloads || writeOffsets) {
payStartFP = payOut.getFilePointer();
level1LastPayFP = level0LastPayFP = payStartFP;
}
}
lastDocID = -1;
level0LastDocID = -1;
level1LastDocID = -1;
this.norms = norms;
if (writeFreqs) {
level0FreqNormAccumulator.clear();
}
}
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
if (docBufferUpto == BLOCK_SIZE) {
flushDocBlock(false);
docBufferUpto = 0;
}
final int docDelta = docID - lastDocID;
if (docID < 0 || docDelta <= 0) {
throw new CorruptIndexException(
"docs out of order (" + docID + " <= " + lastDocID + " )", docOut);
}
docDeltaBuffer[docBufferUpto] = docDelta;
if (writeFreqs) {
freqBuffer[docBufferUpto] = termDocFreq;
}
this.docID = docID;
lastPosition = 0;
lastStartOffset = 0;
if (writeFreqs) {
long norm;
if (fieldHasNorms) {
boolean found = norms.advanceExact(docID);
if (found == false) {
// This can happen if indexing hits a problem after adding a doc to the
// postings but before buffering the norm. Such documents are written
// deleted and will go away on the first merge.
norm = 1L;
} else {
norm = norms.longValue();
assert norm != 0 : docID;
}
} else {
norm = 1L;
}
level0FreqNormAccumulator.add(termDocFreq, norm);
}
}
@Override
public void addPosition(int position, BytesRef payload, int startOffset, int endOffset)
throws IOException {
if (position > IndexWriter.MAX_POSITION) {
throw new CorruptIndexException(
"position="
+ position
+ " is too large (> IndexWriter.MAX_POSITION="
+ IndexWriter.MAX_POSITION
+ ")",
docOut);
}
if (position < 0) {
throw new CorruptIndexException("position=" + position + " is < 0", docOut);
}
posDeltaBuffer[posBufferUpto] = position - lastPosition;
if (writePayloads) {
if (payload == null || payload.length == 0) {
// no payload
payloadLengthBuffer[posBufferUpto] = 0;
} else {
payloadLengthBuffer[posBufferUpto] = payload.length;
if (payloadByteUpto + payload.length > payloadBytes.length) {
payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length);
}
System.arraycopy(
payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length);
payloadByteUpto += payload.length;
}
}
if (writeOffsets) {
assert startOffset >= lastStartOffset;
assert endOffset >= startOffset;
offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset;
offsetLengthBuffer[posBufferUpto] = endOffset - startOffset;
lastStartOffset = startOffset;
}
posBufferUpto++;
lastPosition = position;
if (posBufferUpto == BLOCK_SIZE) {
pforUtil.encode(posDeltaBuffer, posOut);
if (writePayloads) {
pforUtil.encode(payloadLengthBuffer, payOut);
payOut.writeVInt(payloadByteUpto);
payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
payloadByteUpto = 0;
}
if (writeOffsets) {
pforUtil.encode(offsetStartDeltaBuffer, payOut);
pforUtil.encode(offsetLengthBuffer, payOut);
}
posBufferUpto = 0;
}
}
@Override
public void finishDoc() throws IOException {
docBufferUpto++;
docCount++;
lastDocID = docID;
}
/**
* Special vints that are encoded on 2 bytes if they require 15 bits or less. VInt becomes
* especially slow when the number of bytes is variable, so this special layout helps in the case
* when the number likely requires 15 bits or less
*/
static void writeVInt15(DataOutput out, int v) throws IOException {
assert v >= 0;
writeVLong15(out, v);
}
/**
* @see #writeVInt15(DataOutput, int)
*/
static void writeVLong15(DataOutput out, long v) throws IOException {
assert v >= 0;
if ((v & ~0x7FFFL) == 0) {
out.writeShort((short) v);
} else {
out.writeShort((short) (0x8000 | (v & 0x7FFF)));
out.writeVLong(v >> 15);
}
}
private void flushDocBlock(boolean finishTerm) throws IOException {
assert docBufferUpto != 0;
if (docBufferUpto < BLOCK_SIZE) {
assert finishTerm;
PostingsUtil.writeVIntBlock(
level0Output, docDeltaBuffer, freqBuffer, docBufferUpto, writeFreqs);
} else {
if (writeFreqs) {
List<Impact> impacts = level0FreqNormAccumulator.getCompetitiveFreqNormPairs();
if (impacts.size() > maxNumImpactsAtLevel0) {
maxNumImpactsAtLevel0 = impacts.size();
}
writeImpacts(impacts, scratchOutput);
assert level0Output.size() == 0;
if (scratchOutput.size() > maxImpactNumBytesAtLevel0) {
maxImpactNumBytesAtLevel0 = Math.toIntExact(scratchOutput.size());
}
level0Output.writeVLong(scratchOutput.size());
scratchOutput.copyTo(level0Output);
scratchOutput.reset();
if (writePositions) {
level0Output.writeVLong(posOut.getFilePointer() - level0LastPosFP);
level0Output.writeByte((byte) posBufferUpto);
level0LastPosFP = posOut.getFilePointer();
if (writeOffsets || writePayloads) {
level0Output.writeVLong(payOut.getFilePointer() - level0LastPayFP);
level0Output.writeVInt(payloadByteUpto);
level0LastPayFP = payOut.getFilePointer();
}
}
}
long numSkipBytes = level0Output.size();
forDeltaUtil.encodeDeltas(docDeltaBuffer, level0Output);
if (writeFreqs) {
pforUtil.encode(freqBuffer, level0Output);
}
// docID - lastBlockDocID is at least 128, so it can never fit a single byte with a vint
// Even if we subtracted 128, only extremely dense blocks would be eligible to a single byte
// so let's go with 2 bytes right away
writeVInt15(scratchOutput, docID - level0LastDocID);
writeVLong15(scratchOutput, level0Output.size());
numSkipBytes += scratchOutput.size();
level1Output.writeVLong(numSkipBytes);
scratchOutput.copyTo(level1Output);
scratchOutput.reset();
}
level0Output.copyTo(level1Output);
level0Output.reset();
level0LastDocID = docID;
if (writeFreqs) {
level1CompetitiveFreqNormAccumulator.addAll(level0FreqNormAccumulator);
level0FreqNormAccumulator.clear();
}
if ((docCount & LEVEL1_MASK) == 0) { // true every 32 blocks (4,096 docs)
writeLevel1SkipData();
level1LastDocID = docID;
level1CompetitiveFreqNormAccumulator.clear();
} else if (finishTerm) {
level1Output.copyTo(docOut);
level1Output.reset();
level1CompetitiveFreqNormAccumulator.clear();
}
}
private void writeLevel1SkipData() throws IOException {
docOut.writeVInt(docID - level1LastDocID);
long numImpactBytes = scratchOutput.size();
final long level1End;
if (writeFreqs) {
List<Impact> impacts = level1CompetitiveFreqNormAccumulator.getCompetitiveFreqNormPairs();
if (impacts.size() > maxNumImpactsAtLevel1) {
maxNumImpactsAtLevel1 = impacts.size();
}
writeImpacts(impacts, scratchOutput);
numImpactBytes = scratchOutput.size();
if (numImpactBytes > maxImpactNumBytesAtLevel1) {
maxImpactNumBytesAtLevel1 = Math.toIntExact(numImpactBytes);
}
if (writePositions) {
scratchOutput.writeVLong(posOut.getFilePointer() - level1LastPosFP);
scratchOutput.writeByte((byte) posBufferUpto);
level1LastPosFP = posOut.getFilePointer();
if (writeOffsets || writePayloads) {
scratchOutput.writeVLong(payOut.getFilePointer() - level1LastPayFP);
scratchOutput.writeVInt(payloadByteUpto);
level1LastPayFP = payOut.getFilePointer();
}
}
final long level1Len = 2 * Short.BYTES + scratchOutput.size() + level1Output.size();
docOut.writeVLong(level1Len);
level1End = docOut.getFilePointer() + level1Len;
// There are at most 128 impacts, that require at most 2 bytes each
assert numImpactBytes <= Short.MAX_VALUE;
// Like impacts plus a few vlongs, still way under the max short value
assert scratchOutput.size() + Short.BYTES <= Short.MAX_VALUE;
docOut.writeShort((short) (scratchOutput.size() + Short.BYTES));
docOut.writeShort((short) numImpactBytes);
scratchOutput.copyTo(docOut);
scratchOutput.reset();
} else {
docOut.writeVLong(level1Output.size());
level1End = docOut.getFilePointer() + level1Output.size();
}
level1Output.copyTo(docOut);
level1Output.reset();
assert docOut.getFilePointer() == level1End : docOut.getFilePointer() + " " + level1End;
}
static void writeImpacts(Collection<Impact> impacts, DataOutput out) throws IOException {
Impact previous = new Impact(0, 0);
for (Impact impact : impacts) {
assert impact.freq > previous.freq;
assert Long.compareUnsigned(impact.norm, previous.norm) > 0;
int freqDelta = impact.freq - previous.freq - 1;
long normDelta = impact.norm - previous.norm - 1;
if (normDelta == 0) {
// most of time, norm only increases by 1, so we can fold everything in a single byte
out.writeVInt(freqDelta << 1);
} else {
out.writeVInt((freqDelta << 1) | 1);
out.writeZLong(normDelta);
}
previous = impact;
}
}
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(BlockTermState _state) throws IOException {
IntBlockTermState state = (IntBlockTermState) _state;
assert state.docFreq > 0;
// TODO: wasteful we are counting this (counting # docs
// for this term) in two places?
assert state.docFreq == docCount : state.docFreq + " vs " + docCount;
// docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to
// it.
final int singletonDocID;
if (state.docFreq == 1) {
// pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
singletonDocID = (int) docDeltaBuffer[0] - 1;
} else {
singletonDocID = -1;
flushDocBlock(true);
}
final long lastPosBlockOffset;
if (writePositions) {
// totalTermFreq is just total number of positions(or payloads, or offsets)
// associated with current term.
assert state.totalTermFreq != -1;
if (state.totalTermFreq > BLOCK_SIZE) {
// record file offset for last pos in last block
lastPosBlockOffset = posOut.getFilePointer() - posStartFP;
} else {
lastPosBlockOffset = -1;
}
if (posBufferUpto > 0) {
assert posBufferUpto < BLOCK_SIZE;
// TODO: should we send offsets/payloads to
// .pay...? seems wasteful (have to store extra
// vLong for low (< BLOCK_SIZE) DF terms = vast vast
// majority)
// vInt encode the remaining positions/payloads/offsets:
int lastPayloadLength = -1; // force first payload length to be written
int lastOffsetLength = -1; // force first offset length to be written
int payloadBytesReadUpto = 0;
for (int i = 0; i < posBufferUpto; i++) {
final int posDelta = (int) posDeltaBuffer[i];
if (writePayloads) {
final int payloadLength = (int) payloadLengthBuffer[i];
if (payloadLength != lastPayloadLength) {
lastPayloadLength = payloadLength;
posOut.writeVInt((posDelta << 1) | 1);
posOut.writeVInt(payloadLength);
} else {
posOut.writeVInt(posDelta << 1);
}
if (payloadLength != 0) {
posOut.writeBytes(payloadBytes, payloadBytesReadUpto, payloadLength);
payloadBytesReadUpto += payloadLength;
}
} else {
posOut.writeVInt(posDelta);
}
if (writeOffsets) {
int delta = (int) offsetStartDeltaBuffer[i];
int length = (int) offsetLengthBuffer[i];
if (length == lastOffsetLength) {
posOut.writeVInt(delta << 1);
} else {
posOut.writeVInt(delta << 1 | 1);
posOut.writeVInt(length);
lastOffsetLength = length;
}
}
}
if (writePayloads) {
assert payloadBytesReadUpto == payloadByteUpto;
payloadByteUpto = 0;
}
}
} else {
lastPosBlockOffset = -1;
}
state.docStartFP = docStartFP;
state.posStartFP = posStartFP;
state.payStartFP = payStartFP;
state.singletonDocID = singletonDocID;
state.lastPosBlockOffset = lastPosBlockOffset;
docBufferUpto = 0;
posBufferUpto = 0;
lastDocID = -1;
docCount = 0;
}
@Override
public void encodeTerm(
DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute)
throws IOException {
IntBlockTermState state = (IntBlockTermState) _state;
if (absolute) {
lastState = EMPTY_STATE;
assert lastState.docStartFP == 0;
}
if (lastState.singletonDocID != -1
&& state.singletonDocID != -1
&& state.docStartFP == lastState.docStartFP) {
// With runs of rare values such as ID fields, the increment of pointers in the docs file is
// often 0.
// Furthermore some ID schemes like auto-increment IDs or Flake IDs are monotonic, so we
// encode the delta
// between consecutive doc IDs to save space.
final long delta = (long) state.singletonDocID - lastState.singletonDocID;
out.writeVLong((BitUtil.zigZagEncode(delta) << 1) | 0x01);
} else {
out.writeVLong((state.docStartFP - lastState.docStartFP) << 1);
if (state.singletonDocID != -1) {
out.writeVInt(state.singletonDocID);
}
}
if (writePositions) {
out.writeVLong(state.posStartFP - lastState.posStartFP);
if (writePayloads || writeOffsets) {
out.writeVLong(state.payStartFP - lastState.payStartFP);
}
}
if (writePositions) {
if (state.lastPosBlockOffset != -1) {
out.writeVLong(state.lastPosBlockOffset);
}
}
lastState = state;
}
@Override
public void close() throws IOException {
// TODO: add a finish() at least to PushBase? DV too...?
boolean success = false;
try {
if (docOut != null) {
CodecUtil.writeFooter(docOut);
}
if (posOut != null) {
CodecUtil.writeFooter(posOut);
}
if (payOut != null) {
CodecUtil.writeFooter(payOut);
}
if (metaOut != null) {
metaOut.writeInt(maxNumImpactsAtLevel0);
metaOut.writeInt(maxImpactNumBytesAtLevel0);
metaOut.writeInt(maxNumImpactsAtLevel1);
metaOut.writeInt(maxImpactNumBytesAtLevel1);
metaOut.writeLong(docOut.getFilePointer());
if (posOut != null) {
metaOut.writeLong(posOut.getFilePointer());
if (payOut != null) {
metaOut.writeLong(payOut.getFilePointer());
}
}
CodecUtil.writeFooter(metaOut);
}
success = true;
} finally {
if (success) {
IOUtils.close(metaOut, docOut, posOut, payOut);
} else {
IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut);
}
metaOut = docOut = posOut = payOut = null;
}
}
}

View File

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.LongHeap;
import org.apache.lucene.util.packed.PackedInts;
/** Utility class to encode sequences of 128 small positive integers. */
final class PForUtil {
private static final int MAX_EXCEPTIONS = 7;
static boolean allEqual(long[] l) {
for (int i = 1; i < ForUtil.BLOCK_SIZE; ++i) {
if (l[i] != l[0]) {
return false;
}
}
return true;
}
private final ForUtil forUtil;
PForUtil(ForUtil forUtil) {
assert ForUtil.BLOCK_SIZE <= 256 : "blocksize must fit in one byte. got " + ForUtil.BLOCK_SIZE;
this.forUtil = forUtil;
}
/** Encode 128 integers from {@code longs} into {@code out}. */
void encode(long[] longs, DataOutput out) throws IOException {
// Determine the top MAX_EXCEPTIONS + 1 values
final LongHeap top = new LongHeap(MAX_EXCEPTIONS + 1);
for (int i = 0; i <= MAX_EXCEPTIONS; ++i) {
top.push(longs[i]);
}
long topValue = top.top();
for (int i = MAX_EXCEPTIONS + 1; i < ForUtil.BLOCK_SIZE; ++i) {
if (longs[i] > topValue) {
topValue = top.updateTop(longs[i]);
}
}
long max = 0L;
for (int i = 1; i <= top.size(); ++i) {
max = Math.max(max, top.get(i));
}
final int maxBitsRequired = PackedInts.bitsRequired(max);
// We store the patch on a byte, so we can't decrease the number of bits required by more than 8
final int patchedBitsRequired =
Math.max(PackedInts.bitsRequired(topValue), maxBitsRequired - 8);
int numExceptions = 0;
final long maxUnpatchedValue = (1L << patchedBitsRequired) - 1;
for (int i = 2; i <= top.size(); ++i) {
if (top.get(i) > maxUnpatchedValue) {
numExceptions++;
}
}
final byte[] exceptions = new byte[numExceptions * 2];
if (numExceptions > 0) {
int exceptionCount = 0;
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
if (longs[i] > maxUnpatchedValue) {
exceptions[exceptionCount * 2] = (byte) i;
exceptions[exceptionCount * 2 + 1] = (byte) (longs[i] >>> patchedBitsRequired);
longs[i] &= maxUnpatchedValue;
exceptionCount++;
}
}
assert exceptionCount == numExceptions : exceptionCount + " " + numExceptions;
}
if (allEqual(longs) && maxBitsRequired <= 8) {
for (int i = 0; i < numExceptions; ++i) {
exceptions[2 * i + 1] =
(byte) (Byte.toUnsignedLong(exceptions[2 * i + 1]) << patchedBitsRequired);
}
out.writeByte((byte) (numExceptions << 5));
out.writeVLong(longs[0]);
} else {
final int token = (numExceptions << 5) | patchedBitsRequired;
out.writeByte((byte) token);
forUtil.encode(longs, patchedBitsRequired, out);
}
out.writeBytes(exceptions, exceptions.length);
}
/** Decode 128 integers into {@code ints}. */
void decode(PostingIndexInput in, long[] longs) throws IOException {
final int token = Byte.toUnsignedInt(in.in.readByte());
final int bitsPerValue = token & 0x1f;
final int numExceptions = token >>> 5;
if (bitsPerValue == 0) {
Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.in.readVLong());
} else {
in.decode(bitsPerValue, longs);
}
for (int i = 0; i < numExceptions; ++i) {
longs[Byte.toUnsignedInt(in.in.readByte())] |=
Byte.toUnsignedLong(in.in.readByte()) << bitsPerValue;
}
}
/** Skip 128 integers. */
void skip(DataInput in) throws IOException {
final int token = Byte.toUnsignedInt(in.readByte());
final int bitsPerValue = token & 0x1f;
final int numExceptions = token >>> 5;
if (bitsPerValue == 0) {
in.readVLong();
in.skipBytes((numExceptions << 1));
} else {
in.skipBytes(forUtil.numBytes(bitsPerValue) + (numExceptions << 1));
}
}
}

View File

@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import org.apache.lucene.internal.vectorization.PostingDecodingUtil;
import org.apache.lucene.internal.vectorization.VectorizationProvider;
import org.apache.lucene.store.IndexInput;
/**
* Wrapper around an {@link IndexInput} and a {@link ForUtil} that optionally optimizes decoding
* using vectorization.
*/
public final class PostingIndexInput {
private static final VectorizationProvider VECTORIZATION_PROVIDER =
VectorizationProvider.getInstance();
public final IndexInput in;
public final ForUtil forUtil;
private final PostingDecodingUtil postingDecodingUtil;
public PostingIndexInput(IndexInput in, ForUtil forUtil) throws IOException {
this.in = in;
this.forUtil = forUtil;
this.postingDecodingUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(in);
}
/** Decode 128 integers stored on {@code bitsPerValues} bits per value into {@code longs}. */
public void decode(int bitsPerValue, long[] longs) throws IOException {
forUtil.decode(bitsPerValue, in, postingDecodingUtil, longs);
}
/**
* Decode 128 integers stored on {@code bitsPerValues} bits per value, compute their prefix sum,
* and store results into {@code longs}.
*/
public void decodeAndPrefixSum(int bitsPerValue, long base, long[] longs) throws IOException {
forUtil.decodeAndPrefixSum(bitsPerValue, in, postingDecodingUtil, base, longs);
}
}

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
/** Utility class to encode/decode postings block. */
final class PostingsUtil {
/**
* Read values that have been written using variable-length encoding and group-varint encoding
* instead of bit-packing.
*/
static void readVIntBlock(
IndexInput docIn,
long[] docBuffer,
long[] freqBuffer,
int num,
boolean indexHasFreq,
boolean decodeFreq)
throws IOException {
docIn.readGroupVInts(docBuffer, num);
if (indexHasFreq && decodeFreq) {
for (int i = 0; i < num; ++i) {
freqBuffer[i] = docBuffer[i] & 0x01;
docBuffer[i] >>= 1;
if (freqBuffer[i] == 0) {
freqBuffer[i] = docIn.readVInt();
}
}
} else if (indexHasFreq) {
for (int i = 0; i < num; ++i) {
docBuffer[i] >>= 1;
}
}
}
/** Write freq buffer with variable-length encoding and doc buffer with group-varint encoding. */
static void writeVIntBlock(
DataOutput docOut, long[] docBuffer, long[] freqBuffer, int num, boolean writeFreqs)
throws IOException {
if (writeFreqs) {
for (int i = 0; i < num; i++) {
docBuffer[i] = (docBuffer[i] << 1) | (freqBuffer[i] == 1 ? 1 : 0);
}
}
docOut.writeGroupVInts(docBuffer, num);
if (writeFreqs) {
for (int i = 0; i < num; i++) {
final int freq = (int) freqBuffer[i];
if (freq != 1) {
docOut.writeVInt(freq);
}
}
}
}
}

View File

@ -0,0 +1,520 @@
#! /usr/bin/env python
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
from math import gcd
"""Code generation for ForUtil.java"""
MAX_SPECIALIZED_BITS_PER_VALUE = 24
OUTPUT_FILE = "ForUtil.java"
PRIMITIVE_SIZE = [8, 16, 32]
HEADER = """// This file has been automatically generated, DO NOT EDIT
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.lucene912;
import java.io.IOException;
import org.apache.lucene.internal.vectorization.PostingDecodingUtil;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
/**
* Inspired from https://fulmicoton.com/posts/bitpacking/
* Encodes multiple integers in a long to get SIMD-like speedups.
* If bitsPerValue &lt;= 8 then we pack 8 ints per long
* else if bitsPerValue &lt;= 16 we pack 4 ints per long
* else we pack 2 ints per long
*/
public final class ForUtil {
public static final int BLOCK_SIZE = 128;
private static final int BLOCK_SIZE_LOG2 = 7;
private static long expandMask32(long mask32) {
return mask32 | (mask32 << 32);
}
private static long expandMask16(long mask16) {
return expandMask32(mask16 | (mask16 << 16));
}
private static long expandMask8(long mask8) {
return expandMask16(mask8 | (mask8 << 8));
}
private static long mask32(int bitsPerValue) {
return expandMask32((1L << bitsPerValue) - 1);
}
private static long mask16(int bitsPerValue) {
return expandMask16((1L << bitsPerValue) - 1);
}
private static long mask8(int bitsPerValue) {
return expandMask8((1L << bitsPerValue) - 1);
}
private static void expand8(long[] arr) {
for (int i = 0; i < 16; ++i) {
long l = arr[i];
arr[i] = (l >>> 56) & 0xFFL;
arr[16 + i] = (l >>> 48) & 0xFFL;
arr[32 + i] = (l >>> 40) & 0xFFL;
arr[48 + i] = (l >>> 32) & 0xFFL;
arr[64 + i] = (l >>> 24) & 0xFFL;
arr[80 + i] = (l >>> 16) & 0xFFL;
arr[96 + i] = (l >>> 8) & 0xFFL;
arr[112 + i] = l & 0xFFL;
}
}
private static void expand8To32(long[] arr) {
for (int i = 0; i < 16; ++i) {
long l = arr[i];
arr[i] = (l >>> 24) & 0x000000FF000000FFL;
arr[16 + i] = (l >>> 16) & 0x000000FF000000FFL;
arr[32 + i] = (l >>> 8) & 0x000000FF000000FFL;
arr[48 + i] = l & 0x000000FF000000FFL;
}
}
private static void collapse8(long[] arr) {
for (int i = 0; i < 16; ++i) {
arr[i] =
(arr[i] << 56)
| (arr[16 + i] << 48)
| (arr[32 + i] << 40)
| (arr[48 + i] << 32)
| (arr[64 + i] << 24)
| (arr[80 + i] << 16)
| (arr[96 + i] << 8)
| arr[112 + i];
}
}
private static void expand16(long[] arr) {
for (int i = 0; i < 32; ++i) {
long l = arr[i];
arr[i] = (l >>> 48) & 0xFFFFL;
arr[32 + i] = (l >>> 32) & 0xFFFFL;
arr[64 + i] = (l >>> 16) & 0xFFFFL;
arr[96 + i] = l & 0xFFFFL;
}
}
private static void expand16To32(long[] arr) {
for (int i = 0; i < 32; ++i) {
long l = arr[i];
arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL;
arr[32 + i] = l & 0x0000FFFF0000FFFFL;
}
}
private static void collapse16(long[] arr) {
for (int i = 0; i < 32; ++i) {
arr[i] = (arr[i] << 48) | (arr[32 + i] << 32) | (arr[64 + i] << 16) | arr[96 + i];
}
}
private static void expand32(long[] arr) {
for (int i = 0; i < 64; ++i) {
long l = arr[i];
arr[i] = l >>> 32;
arr[64 + i] = l & 0xFFFFFFFFL;
}
}
private static void collapse32(long[] arr) {
for (int i = 0; i < 64; ++i) {
arr[i] = (arr[i] << 32) | arr[64 + i];
}
}
private static void prefixSum8(long[] arr, long base) {
expand8To32(arr);
prefixSum32(arr, base);
}
private static void prefixSum16(long[] arr, long base) {
// We need to move to the next primitive size to avoid overflows
expand16To32(arr);
prefixSum32(arr, base);
}
private static void prefixSum32(long[] arr, long base) {
arr[0] += base << 32;
innerPrefixSum32(arr);
expand32(arr);
final long l = arr[BLOCK_SIZE/2-1];
for (int i = BLOCK_SIZE/2; i < BLOCK_SIZE; ++i) {
arr[i] += l;
}
}
// For some reason unrolling seems to help
private static void innerPrefixSum32(long[] arr) {
arr[1] += arr[0];
arr[2] += arr[1];
arr[3] += arr[2];
arr[4] += arr[3];
arr[5] += arr[4];
arr[6] += arr[5];
arr[7] += arr[6];
arr[8] += arr[7];
arr[9] += arr[8];
arr[10] += arr[9];
arr[11] += arr[10];
arr[12] += arr[11];
arr[13] += arr[12];
arr[14] += arr[13];
arr[15] += arr[14];
arr[16] += arr[15];
arr[17] += arr[16];
arr[18] += arr[17];
arr[19] += arr[18];
arr[20] += arr[19];
arr[21] += arr[20];
arr[22] += arr[21];
arr[23] += arr[22];
arr[24] += arr[23];
arr[25] += arr[24];
arr[26] += arr[25];
arr[27] += arr[26];
arr[28] += arr[27];
arr[29] += arr[28];
arr[30] += arr[29];
arr[31] += arr[30];
arr[32] += arr[31];
arr[33] += arr[32];
arr[34] += arr[33];
arr[35] += arr[34];
arr[36] += arr[35];
arr[37] += arr[36];
arr[38] += arr[37];
arr[39] += arr[38];
arr[40] += arr[39];
arr[41] += arr[40];
arr[42] += arr[41];
arr[43] += arr[42];
arr[44] += arr[43];
arr[45] += arr[44];
arr[46] += arr[45];
arr[47] += arr[46];
arr[48] += arr[47];
arr[49] += arr[48];
arr[50] += arr[49];
arr[51] += arr[50];
arr[52] += arr[51];
arr[53] += arr[52];
arr[54] += arr[53];
arr[55] += arr[54];
arr[56] += arr[55];
arr[57] += arr[56];
arr[58] += arr[57];
arr[59] += arr[58];
arr[60] += arr[59];
arr[61] += arr[60];
arr[62] += arr[61];
arr[63] += arr[62];
}
private final long[] tmp = new long[BLOCK_SIZE / 2];
/** Encode 128 integers from {@code longs} into {@code out}. */
void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException {
final int nextPrimitive;
final int numLongs;
if (bitsPerValue <= 8) {
nextPrimitive = 8;
numLongs = BLOCK_SIZE / 8;
collapse8(longs);
} else if (bitsPerValue <= 16) {
nextPrimitive = 16;
numLongs = BLOCK_SIZE / 4;
collapse16(longs);
} else {
nextPrimitive = 32;
numLongs = BLOCK_SIZE / 2;
collapse32(longs);
}
final int numLongsPerShift = bitsPerValue * 2;
int idx = 0;
int shift = nextPrimitive - bitsPerValue;
for (int i = 0; i < numLongsPerShift; ++i) {
tmp[i] = longs[idx++] << shift;
}
for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) {
for (int i = 0; i < numLongsPerShift; ++i) {
tmp[i] |= longs[idx++] << shift;
}
}
final int remainingBitsPerLong = shift + bitsPerValue;
final long maskRemainingBitsPerLong;
if (nextPrimitive == 8) {
maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong];
} else if (nextPrimitive == 16) {
maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong];
} else {
maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong];
}
int tmpIdx = 0;
int remainingBitsPerValue = bitsPerValue;
while (idx < numLongs) {
if (remainingBitsPerValue >= remainingBitsPerLong) {
remainingBitsPerValue -= remainingBitsPerLong;
tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong;
if (remainingBitsPerValue == 0) {
idx++;
remainingBitsPerValue = bitsPerValue;
}
} else {
final long mask1, mask2;
if (nextPrimitive == 8) {
mask1 = MASKS8[remainingBitsPerValue];
mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue];
} else if (nextPrimitive == 16) {
mask1 = MASKS16[remainingBitsPerValue];
mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue];
} else {
mask1 = MASKS32[remainingBitsPerValue];
mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue];
}
tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue);
remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue;
tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2;
}
}
for (int i = 0; i < numLongsPerShift; ++i) {
out.writeLong(tmp[i]);
}
}
/** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */
int numBytes(int bitsPerValue) {
return bitsPerValue << (BLOCK_SIZE_LOG2 - 3);
}
private static void decodeSlow(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs)
throws IOException {
final int numLongs = bitsPerValue << 1;
final long mask = MASKS32[bitsPerValue];
pdu.splitLongs(numLongs, longs, 32 - bitsPerValue, mask, tmp, 0, -1L);
int longsIdx = numLongs;
int shift = 32 - 2 * bitsPerValue;
for (; shift >= 0; shift -= bitsPerValue) {
shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask);
longsIdx += numLongs;
}
final int remainingBitsPerLong = shift + bitsPerValue;
final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong];
int tmpIdx = 0;
int remainingBits = remainingBitsPerLong;
for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) {
int b = bitsPerValue - remainingBits;
long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b;
while (b >= remainingBitsPerLong) {
b -= remainingBitsPerLong;
l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b;
}
if (b > 0) {
l |= (tmp[tmpIdx] >>> (remainingBitsPerLong - b)) & MASKS32[b];
remainingBits = remainingBitsPerLong - b;
} else {
remainingBits = remainingBitsPerLong;
}
longs[longsIdx] = l;
}
}
/**
* The pattern that this shiftLongs method applies is recognized by the C2 compiler, which
* generates SIMD instructions for it in order to shift multiple longs at once.
*/
private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) {
for (int i = 0; i < count; ++i) {
b[bi + i] = (a[i] >>> shift) & mask;
}
}
/**
* Likewise, but for a simple mask.
*/
private static void maskLongs(long[] a, int count, long[] b, int bi, long mask) {
for (int i = 0; i < count; ++i) {
b[bi + i] = a[i] & mask;
}
}
"""
def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f):
iteration = 1
num_longs = bpv * num_values / remaining_bits_per_long
while num_longs % 2 == 0 and num_values % 2 == 0:
num_longs /= 2
num_values /= 2
iteration *= 2
f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values))
i = 0
remaining_bits = 0
tmp_idx = 0
for i in range(int(num_values)):
b = bpv
if remaining_bits == 0:
b -= remaining_bits_per_long
f.write(' long l%d = tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b))
else:
b -= remaining_bits
f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b))
tmp_idx += 1
while b >= remaining_bits_per_long:
b -= remaining_bits_per_long
f.write(' l%d |= tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b))
tmp_idx += 1
if b > 0:
f.write(' l%d |= (tmp[tmpIdx + %d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_long-b, next_primitive, b))
remaining_bits = remaining_bits_per_long-b
f.write(' longs[longsIdx + %d] = l%d;\n' %(i, i))
f.write(' }\n')
def writeDecode(bpv, f):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' private static void decode%d(IndexInput in, PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %bpv)
if bpv == next_primitive:
f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2))
elif bpv * 2 == next_primitive:
f.write(' pdu.splitLongs(%d, longs, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, next_primitive, bpv, bpv*2, next_primitive, next_primitive - bpv))
else:
num_values_per_long = 64 / next_primitive
f.write(' pdu.splitLongs(%d, longs, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, next_primitive, bpv, next_primitive, next_primitive - bpv))
shift = next_primitive - 2 * bpv
o = 2 * bpv
while shift >= 0:
if shift == 0:
f.write(' maskLongs(tmp, %d, longs, %d, MASK%d_%d);\n' %(bpv*2, o, next_primitive, bpv))
else:
f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv))
o += bpv*2
shift -= bpv
remaining_bits = shift + bpv
if remaining_bits > 0:
if remaining_bits != next_primitive - bpv:
# values in tmp still have more bits per value than remaining_bits, clear the higher bits now
f.write(' maskLongs(tmp, %d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive, remaining_bits))
writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_long - o, f)
f.write(' }\n')
if __name__ == '__main__':
f = open(OUTPUT_FILE, 'w')
f.write(HEADER)
for primitive_size in PRIMITIVE_SIZE:
f.write(' private static final long[] MASKS%d = new long[%d];\n' %(primitive_size, primitive_size))
f.write('\n')
f.write(' static {\n')
for primitive_size in PRIMITIVE_SIZE:
f.write(' for (int i = 0; i < %d; ++i) {\n' %primitive_size)
f.write(' MASKS%d[i] = mask%d(i);\n' %(primitive_size, primitive_size))
f.write(' }\n')
f.write(' }')
f.write("""
// mark values in array as final longs to avoid the cost of reading array, arrays should only be
// used when the idx is a variable
""")
for primitive_size in PRIMITIVE_SIZE:
for bpv in range(1, min(MAX_SPECIALIZED_BITS_PER_VALUE + 1, primitive_size)):
if bpv * 2 != primitive_size or primitive_size == 8:
f.write(' private static final long MASK%d_%d = MASKS%d[%d];\n' %(primitive_size, bpv, primitive_size, bpv))
f.write("""
/** Decode 128 integers into {@code longs}. */
void decode(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long[] longs) throws IOException {
switch (bitsPerValue) {
""")
for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' case %d:\n' %bpv)
f.write(' decode%d(in, pdu, tmp, longs);\n' %bpv)
f.write(' expand%d(longs);\n' %next_primitive)
f.write(' break;\n')
f.write(' default:\n')
f.write(' decodeSlow(bitsPerValue, in, pdu, tmp, longs);\n')
f.write(' expand32(longs);\n')
f.write(' break;\n')
f.write(' }\n')
f.write(' }\n')
f.write("""
/**
* Delta-decode 128 integers into {@code longs}.
*/
void decodeAndPrefixSum(int bitsPerValue, IndexInput in, PostingDecodingUtil pdu, long base, long[] longs) throws IOException {
switch (bitsPerValue) {
""")
for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' case %d:\n' %bpv)
f.write(' decode%d(in, pdu, tmp, longs);\n' %bpv)
f.write(' prefixSum%d(longs, base);\n' %next_primitive)
f.write(' break;\n')
f.write(' default:\n')
f.write(' decodeSlow(bitsPerValue, in, pdu, tmp, longs);\n')
f.write(' prefixSum32(longs, base);\n')
f.write(' break;\n')
f.write(' }\n')
f.write(' }\n')
f.write('\n')
for i in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
writeDecode(i, f)
if i < MAX_SPECIALIZED_BITS_PER_VALUE:
f.write('\n')
f.write('}\n')

View File

@ -0,0 +1,431 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Lucene 9.12 file format.
*
* <h2>Apache Lucene - Index File Formats</h2>
*
* <div>
*
* <ul>
* <li><a href="#Introduction">Introduction</a>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a>
* <li><a href="#Types_of_Fields">Types of Fields</a>
* <li><a href="#Segments">Segments</a>
* <li><a href="#Document_Numbers">Document Numbers</a>
* </ul>
* <li><a href="#Overview">Index Structure Overview</a>
* <li><a href="#File_Naming">File Naming</a>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a>
* <li><a href="#History">History</a>
* <li><a href="#Limitations">Limitations</a>
* </ul>
* </ul>
*
* </div> <a id="Introduction"></a>
*
* <h3>Introduction</h3>
*
* <div>
*
* <p>This document defines the index file formats used in this version of Lucene. If you are using
* a different version of Lucene, please consult the copy of <code>docs/</code> that was distributed
* with the version you are using.
*
* <p>This document attempts to provide a high-level definition of the Apache Lucene file formats.
* </div> <a id="Definitions"></a>
*
* <h3>Definitions</h3>
*
* <div>
*
* <p>The fundamental concepts in Lucene are index, document, field and term.
*
* <p>An index contains a sequence of documents.
*
* <ul>
* <li>A document is a sequence of fields.
* <li>A field is a named sequence of terms.
* <li>A term is a sequence of bytes.
* </ul>
*
* <p>The same sequence of bytes in two different fields is considered a different term. Thus terms
* are represented as a pair: the string naming the field, and the bytes within the field. <a
* id="Inverted_Indexing"></a>
*
* <h4>Inverted Indexing</h4>
*
* <p>Lucene's index stores terms and statistics about those terms in order to make term-based
* search more efficient. Lucene's terms index falls into the family of indexes known as an
* <i>inverted index.</i> This is because it can list, for a term, the documents that contain it.
* This is the inverse of the natural relationship, in which documents list terms. <a
* id="Types_of_Fields"></a>
*
* <h4>Types of Fields</h4>
*
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored in the index
* literally, in a non-inverted manner. Fields that are inverted are called <i>indexed</i>. A field
* may be both stored and indexed.
*
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the text of a field
* may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is
* useful for certain identifier fields to be indexed literally.
*
* <p>See the {@link org.apache.lucene.document.Field Field} java docs for more information on
* Fields. <a id="Segments"></a>
*
* <h4>Segments</h4>
*
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>. Each segment is a
* fully independent index, which could be searched separately. Indexes evolve by:
*
* <ol>
* <li>Creating new segments for newly added documents.
* <li>Merging existing segments.
* </ol>
*
* <p>Searches may involve multiple segments and/or multiple indexes, each index potentially
* composed of a set of segments. <a id="Document_Numbers"></a>
*
* <h4>Document Numbers</h4>
*
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>. The first
* document added to an index is numbered zero, and each subsequent document added gets a number one
* greater than the previous.
*
* <p>Note that a document's number may change, so caution should be taken when storing these
* numbers outside of Lucene. In particular, numbers may change in the following situations:
*
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and must be
* converted before they can be used in a larger context. The standard technique is to
* allocate each segment a range of values, based on the range of numbers used in that
* segment. To convert a document number from a segment to an external value, the segment's
* <i>base</i> document number is added. To convert an external value back to a
* segment-specific value, the segment is identified by the range that the external value is
* in, and the segment's base value is subtracted. For example two five document segments
* might be combined, so that the first segment has a base value of zero, and the second of
* five. Document three from the second segment would have an external value of eight.
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are eventually
* removed as the index evolves through merging. Deleted documents are dropped when segments
* are merged. A freshly-merged segment thus has no gaps in its numbering.
* </ul>
*
* </div> <a id="Overview"></a>
*
* <h3>Index Structure Overview</h3>
*
* <div>
*
* <p>Each segment index maintains the following:
*
* <ul>
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This
* contains metadata about a segment, such as the number of documents, what files it uses, and
* information about how the segment is sorted
* <li>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This
* contains metadata about the set of named fields used in the index.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes are
* field names. These are used to store auxiliary information about the document, such as its
* title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* <li>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term dictionary}. A
* dictionary containing all of the terms used in all of the indexed fields of all of the
* documents. The dictionary also contains the number of documents which contain the term, and
* pointers to the term's frequency and proximity data.
* <li>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Frequency data}. For
* each term in the dictionary, the numbers of all the documents that contain that term, and
* the frequency of the term in that document, unless frequencies are omitted ({@link
* org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
* <li>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Proximity data}. For
* each term in the dictionary, the positions that the term occurs in each document. Note that
* this will not exist if all fields in all documents omit position data.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
* each field in each document, a value is stored that is multiplied into the score for hits
* on that field.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
* field in each document, the term vector (sometimes called document vector) may be stored. A
* term vector consists of term text and term frequency. To add Term Vectors to your index see
* the {@link org.apache.lucene.document.Field Field} constructors
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like
* stored values, these are also keyed by document number, but are generally intended to be
* loaded into main memory for fast access. Whereas stored values are generally intended for
* summary results from searches, per-document values are useful for things like scoring
* factors.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An
* optional file indicating which documents are live.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair
* of files, recording dimensionally indexed fields, to enable fast numeric range filtering
* and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
* intersection (2D, 3D).
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The
* vector format stores numeric vectors in a format optimized for random access and
* computation, supporting high-dimensional nearest-neighbor search.
* </ul>
*
* <p>Details on each of these are provided in their linked pages. </div> <a id="File_Naming"></a>
*
* <h3>File Naming</h3>
*
* <div>
*
* <p>All files belonging to a segment have the same name with varying extensions. The extensions
* correspond to the different file formats described below. When using the Compound File format
* (default for small segments) these files (except for the Segment info file, the Lock file, and
* Deleted documents file) are collapsed into a single .cfs file (see below for details)
*
* <p>Typically, all segments in an index are stored in a single directory, although this is not
* required.
*
* <p>File names are never re-used. That is, when any file is saved to the Directory it is given a
* never before used filename. This is achieved using a simple generations approach. For example,
* the first segments file is segments_1, then segments_2, etc. The generation is a sequential long
* integer represented in alpha-numeric (base 36) form. </div> <a id="file-names"></a>
*
* <h3>Summary of File Extensions</h3>
*
* <div>
*
* <p>The following table summarizes the names and extensions of the files in Lucene:
*
* <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
* <caption>lucene filenames by extension</caption>
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what documents are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}</td>
* <td>.vec, .vem, .veq, vex</td>
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data,
* <code>.vem</code> the vector metadata, <code>.veq</code> the quantized vector data, and <code>.vex</code> the
* hnsw graph data.</td>
* </tr>
* </table>
*
* </div> <a id="Lock_File"></a>
*
* <h3>Lock File</h3>
*
* The write lock, which is stored in the index directory by default, is named "write.lock". If the
* lock directory is different from the index directory then the write lock will be named
* "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index
* directory. When this file is present, a writer is currently modifying the index (adding or
* removing documents). This lock file ensures that only one writer is modifying the index at a
* time. <a id="History"></a>
*
* <h3>History</h3>
*
* <p>Compatibility notes are provided in this document, describing how file formats have changed
* from prior versions:
*
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit
* lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching
* or adding/deleting of docs. When the new segments file is saved (committed), it will be
* written in the new file format (meaning no specific "upgrade" process is needed). But note
* that once a commit has occurred, pre-2.1 Lucene will not be able to read the index.
* <li>In version 2.3, the file format was changed to allow segments to share a single set of doc
* store (vectors &amp; stored fields) files. This allows for faster indexing in certain
* cases. The change is fully backwards compatible (in the same way as the lock-less commits
* change in 2.1).
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified
* UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">LUCENE-510</a> for
* details.
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData may be passed to
* IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N
* file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">LUCENE-1382</a> for
* details. Also, diagnostics were added to each segment written recording details about why
* it was written (due to flush, merge; which OS/JRE was used; etc.). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.
* <li>In version 3.0, compressed fields are no longer written to the index (they can still be
* read, but on merge the new segment will write them, uncompressed). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> for details.
* <li>In version 3.1, segments records the code version that created them. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> for details.
* <li>In version 3.2, numeric fields are written as natively to stored fields file, previously
* they were stored in text format only.
* <li>In version 3.4, fields can omit position data while still indexing term frequencies.
* <li>In version 4.0, the format of the inverted index became extensible via the {@link
* org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues})
* was introduced. Normalization factors need no longer be a single byte, they can be any
* {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be
* unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into
* the postings lists. Payloads can be stored in the term vectors.
* <li>In version 4.1, the format of the postings list changed to use either of FOR compression or
* variable-byte encoding, depending upon the frequency of the term. Terms appearing only once
* were changed to inline directly into the term dictionary. Stored fields are compressed by
* default.
* <li>In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued
* type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields.
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.
* <li>In version 4.8, checksum footers were added to the end of each index file for improved data
* integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32
* checksum of the file.
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is
* suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk: addresses
* for binary fields and ord indexes for multi-valued fields.
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
* <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index
* sorting.
* <li>In version 7.0, DocValues have been improved to better support sparse doc values thanks to
* an iterator API.
* <li>In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term
* freq, normalization factor) pairs that may trigger the maximum score of the block. This
* information is recorded alongside skip data in order to be able to skip blocks of doc ids
* if they may not produce high enough scores. Additionally doc values and norms has been
* extended with jump-tables to make access O(1) instead of O(n), where n is the number of
* elements to skip when advancing in the data.
* <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
* performant encoding that is vectorized.
* <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to allow
* user-defined sorts to be used
* <li>In version 8.7, stored fields compression became adaptive to better handle documents with
* smaller stored fields.
* <li>In version 9.0, vector-valued fields were added.
* <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
* <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
* IndexDISI. ordToDoc mappings was added to .vem.
* <li>In version 9.5, HNSW graph connections were changed to be delta-encoded with vints.
* Additionally, metadata file size improvements were made by delta-encoding nodes by graph
* layer and not writing the node ids for the zeroth layer.
* <li>In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector
* format to utilize int8 quantized vectors for float32 vector search.
* <li>In version 9.12, skip data was refactored to have only two levels: every 128 docs and every
* 4,06 docs, and to be inlined in postings lists. This resulted in a speedup for queries that
* need skipping, especially conjunctions.
* </ul>
*
* <a id="Limitations"></a>
*
* <h3>Limitations</h3>
*
* <div>
*
* <p>Lucene uses a Java <code>int</code> to refer to document numbers, and the index file format
* uses an <code>Int32</code> on-disk to store document numbers. This is a limitation of both the
* index file format and the current implementation. Eventually these should be replaced with either
* <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
* VInt} values which have no limit. </div>
*/
package org.apache.lucene.codecs.lucene912;

View File

@ -68,10 +68,10 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
private static final long SHALLOW_SIZE =
RamUsageEstimator.shallowSizeOfInstance(Lucene99HnswVectorsFormat.class);
private final FlatVectorsReader flatVectorsReader;
private final FieldInfos fieldInfos;
private final Map<String, FieldEntry> fields = new HashMap<>();
private final IndexInput vectorIndex;
private final FlatVectorsReader flatVectorsReader;
public Lucene99HnswVectorsReader(SegmentReadState state, FlatVectorsReader flatVectorsReader)
throws IOException {
@ -93,13 +93,13 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
Lucene99HnswVectorsFormat.VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
readFields(meta, state.fieldInfos);
readFields(meta);
} catch (Throwable exception) {
priorE = exception;
} finally {
CodecUtil.checkFooter(meta, priorE);
}
vectorIndex =
this.vectorIndex =
openDataInput(
state,
versionMeta,
@ -154,9 +154,9 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
}
}
private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
private void readFields(ChecksumIndexInput meta) throws IOException {
for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
FieldInfo info = infos.fieldInfo(fieldNumber);
FieldInfo info = fieldInfos.fieldInfo(fieldNumber);
if (info == null) {
throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
}
@ -463,6 +463,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
// unsafe; no bounds checking
dataIn.seek(graphLevelNodeOffsets.get(targetIndex + graphLevelNodeIndexOffsets[level]));
arcCount = dataIn.readVInt();
assert arcCount <= currentNeighborsBuffer.length : "too many neighbors: " + arcCount;
if (arcCount > 0) {
currentNeighborsBuffer[0] = dataIn.readVInt();
for (int i = 1; i < arcCount; i++) {

View File

@ -612,10 +612,10 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
throw new UnsupportedOperationException();
}
OnHeapHnswGraph getGraph() {
OnHeapHnswGraph getGraph() throws IOException {
assert flatFieldVectorsWriter.isFinished();
if (node > 0) {
return hnswGraphBuilder.getGraph();
return hnswGraphBuilder.getCompletedGraph();
} else {
return null;
}

View File

@ -677,6 +677,10 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
Float confidenceInterval,
byte bits)
throws IOException {
if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) {
floatVectorValues = new NormalizedFloatVectorValues(floatVectorValues);
vectorSimilarityFunction = VectorSimilarityFunction.DOT_PRODUCT;
}
if (confidenceInterval != null && confidenceInterval == DYNAMIC_CONFIDENCE_INTERVAL) {
return ScalarQuantizer.fromVectorsAutoInterval(
floatVectorValues, vectorSimilarityFunction, numVectors, bits);
@ -797,10 +801,9 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
if (floatVectors.size() == 0) {
return new ScalarQuantizer(0, 0, bits);
}
FloatVectorValues floatVectorValues = new FloatVectorWrapper(floatVectors, normalize);
ScalarQuantizer quantizer =
buildScalarQuantizer(
floatVectorValues,
new FloatVectorWrapper(floatVectors),
floatVectors.size(),
fieldInfo.getVectorSimilarityFunction(),
confidenceInterval,
@ -851,14 +854,10 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
static class FloatVectorWrapper extends FloatVectorValues {
private final List<float[]> vectorList;
private final float[] copy;
private final boolean normalize;
protected int curDoc = -1;
FloatVectorWrapper(List<float[]> vectorList, boolean normalize) {
FloatVectorWrapper(List<float[]> vectorList) {
this.vectorList = vectorList;
this.copy = new float[vectorList.get(0).length];
this.normalize = normalize;
}
@Override
@ -876,11 +875,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
if (curDoc == -1 || curDoc >= vectorList.size()) {
throw new IOException("Current doc not set or too many iterations");
}
if (normalize) {
System.arraycopy(vectorList.get(curDoc), 0, copy, 0, copy.length);
VectorUtil.l2normalize(copy);
return copy;
}
return vectorList.get(curDoc);
}
@ -949,13 +943,16 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
// quantization?
|| scalarQuantizer.getBits() <= 4
|| shouldRequantize(reader.getQuantizationState(fieldInfo.name), scalarQuantizer)) {
FloatVectorValues toQuantize =
mergeState.knnVectorsReaders[i].getFloatVectorValues(fieldInfo.name);
if (fieldInfo.getVectorSimilarityFunction() == VectorSimilarityFunction.COSINE) {
toQuantize = new NormalizedFloatVectorValues(toQuantize);
}
sub =
new QuantizedByteVectorValueSub(
mergeState.docMaps[i],
new QuantizedFloatVectorValues(
mergeState.knnVectorsReaders[i].getFloatVectorValues(fieldInfo.name),
fieldInfo.getVectorSimilarityFunction(),
scalarQuantizer));
toQuantize, fieldInfo.getVectorSimilarityFunction(), scalarQuantizer));
} else {
sub =
new QuantizedByteVectorValueSub(
@ -1042,7 +1039,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
private final FloatVectorValues values;
private final ScalarQuantizer quantizer;
private final byte[] quantizedVector;
private final float[] normalizedVector;
private float offsetValue = 0f;
private final VectorSimilarityFunction vectorSimilarityFunction;
@ -1055,11 +1051,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
this.quantizer = quantizer;
this.quantizedVector = new byte[values.dimension()];
this.vectorSimilarityFunction = vectorSimilarityFunction;
if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) {
this.normalizedVector = new float[values.dimension()];
} else {
this.normalizedVector = null;
}
}
@Override
@ -1111,15 +1102,8 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
}
private void quantize() throws IOException {
if (vectorSimilarityFunction == VectorSimilarityFunction.COSINE) {
System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length);
VectorUtil.l2normalize(normalizedVector);
offsetValue =
quantizer.quantize(normalizedVector, quantizedVector, vectorSimilarityFunction);
} else {
offsetValue =
quantizer.quantize(values.vectorValue(), quantizedVector, vectorSimilarityFunction);
}
offsetValue =
quantizer.quantize(values.vectorValue(), quantizedVector, vectorSimilarityFunction);
}
}
@ -1216,4 +1200,60 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
throw new UnsupportedOperationException();
}
}
static final class NormalizedFloatVectorValues extends FloatVectorValues {
private final FloatVectorValues values;
private final float[] normalizedVector;
int curDoc = -1;
public NormalizedFloatVectorValues(FloatVectorValues values) {
this.values = values;
this.normalizedVector = new float[values.dimension()];
}
@Override
public int dimension() {
return values.dimension();
}
@Override
public int size() {
return values.size();
}
@Override
public float[] vectorValue() throws IOException {
return normalizedVector;
}
@Override
public VectorScorer scorer(float[] query) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int docID() {
return values.docID();
}
@Override
public int nextDoc() throws IOException {
curDoc = values.nextDoc();
if (curDoc != NO_MORE_DOCS) {
System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length);
VectorUtil.l2normalize(normalizedVector);
}
return curDoc;
}
@Override
public int advance(int target) throws IOException {
curDoc = values.advance(target);
if (curDoc != NO_MORE_DOCS) {
System.arraycopy(values.vectorValue(), 0, normalizedVector, 0, normalizedVector.length);
VectorUtil.l2normalize(normalizedVector);
}
return curDoc;
}
}
}

View File

@ -15,414 +15,5 @@
* limitations under the License.
*/
/**
* Lucene 9.9 file format.
*
* <h2>Apache Lucene - Index File Formats</h2>
*
* <div>
*
* <ul>
* <li><a href="#Introduction">Introduction</a>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a>
* <li><a href="#Types_of_Fields">Types of Fields</a>
* <li><a href="#Segments">Segments</a>
* <li><a href="#Document_Numbers">Document Numbers</a>
* </ul>
* <li><a href="#Overview">Index Structure Overview</a>
* <li><a href="#File_Naming">File Naming</a>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a>
* <li><a href="#History">History</a>
* <li><a href="#Limitations">Limitations</a>
* </ul>
* </ul>
*
* </div> <a id="Introduction"></a>
*
* <h3>Introduction</h3>
*
* <div>
*
* <p>This document defines the index file formats used in this version of Lucene. If you are using
* a different version of Lucene, please consult the copy of <code>docs/</code> that was distributed
* with the version you are using.
*
* <p>This document attempts to provide a high-level definition of the Apache Lucene file formats.
* </div> <a id="Definitions"></a>
*
* <h3>Definitions</h3>
*
* <div>
*
* <p>The fundamental concepts in Lucene are index, document, field and term.
*
* <p>An index contains a sequence of documents.
*
* <ul>
* <li>A document is a sequence of fields.
* <li>A field is a named sequence of terms.
* <li>A term is a sequence of bytes.
* </ul>
*
* <p>The same sequence of bytes in two different fields is considered a different term. Thus terms
* are represented as a pair: the string naming the field, and the bytes within the field. <a
* id="Inverted_Indexing"></a>
*
* <h4>Inverted Indexing</h4>
*
* <p>Lucene's index stores terms and statistics about those terms in order to make term-based
* search more efficient. Lucene's terms index falls into the family of indexes known as an
* <i>inverted index.</i> This is because it can list, for a term, the documents that contain it.
* This is the inverse of the natural relationship, in which documents list terms. <a
* id="Types_of_Fields"></a>
*
* <h4>Types of Fields</h4>
*
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored in the index
* literally, in a non-inverted manner. Fields that are inverted are called <i>indexed</i>. A field
* may be both stored and indexed.
*
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the text of a field
* may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is
* useful for certain identifier fields to be indexed literally.
*
* <p>See the {@link org.apache.lucene.document.Field Field} java docs for more information on
* Fields. <a id="Segments"></a>
*
* <h4>Segments</h4>
*
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>. Each segment is a
* fully independent index, which could be searched separately. Indexes evolve by:
*
* <ol>
* <li>Creating new segments for newly added documents.
* <li>Merging existing segments.
* </ol>
*
* <p>Searches may involve multiple segments and/or multiple indexes, each index potentially
* composed of a set of segments. <a id="Document_Numbers"></a>
*
* <h4>Document Numbers</h4>
*
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>. The first
* document added to an index is numbered zero, and each subsequent document added gets a number one
* greater than the previous.
*
* <p>Note that a document's number may change, so caution should be taken when storing these
* numbers outside of Lucene. In particular, numbers may change in the following situations:
*
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and must be
* converted before they can be used in a larger context. The standard technique is to
* allocate each segment a range of values, based on the range of numbers used in that
* segment. To convert a document number from a segment to an external value, the segment's
* <i>base</i> document number is added. To convert an external value back to a
* segment-specific value, the segment is identified by the range that the external value is
* in, and the segment's base value is subtracted. For example two five document segments
* might be combined, so that the first segment has a base value of zero, and the second of
* five. Document three from the second segment would have an external value of eight.
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are eventually
* removed as the index evolves through merging. Deleted documents are dropped when segments
* are merged. A freshly-merged segment thus has no gaps in its numbering.
* </ul>
*
* </div> <a id="Overview"></a>
*
* <h3>Index Structure Overview</h3>
*
* <div>
*
* <p>Each segment index maintains the following:
*
* <ul>
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This
* contains metadata about a segment, such as the number of documents, what files it uses, and
* information about how the segment is sorted
* <li>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This
* contains metadata about the set of named fields used in the index.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes are
* field names. These are used to store auxiliary information about the document, such as its
* title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term dictionary}. A
* dictionary containing all of the terms used in all of the indexed fields of all of the
* documents. The dictionary also contains the number of documents which contain the term, and
* pointers to the term's frequency and proximity data.
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Frequency data}. For
* each term in the dictionary, the numbers of all the documents that contain that term, and
* the frequency of the term in that document, unless frequencies are omitted ({@link
* org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Proximity data}. For
* each term in the dictionary, the positions that the term occurs in each document. Note that
* this will not exist if all fields in all documents omit position data.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For
* each field in each document, a value is stored that is multiplied into the score for hits
* on that field.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each
* field in each document, the term vector (sometimes called document vector) may be stored. A
* term vector consists of term text and term frequency. To add Term Vectors to your index see
* the {@link org.apache.lucene.document.Field Field} constructors
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like
* stored values, these are also keyed by document number, but are generally intended to be
* loaded into main memory for fast access. Whereas stored values are generally intended for
* summary results from searches, per-document values are useful for things like scoring
* factors.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An
* optional file indicating which documents are live.
* <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair
* of files, recording dimensionally indexed fields, to enable fast numeric range filtering
* and large numeric values like BigInteger and BigDecimal (1D) and geographic shape
* intersection (2D, 3D).
* <li>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The
* vector format stores numeric vectors in a format optimized for random access and
* computation, supporting high-dimensional nearest-neighbor search.
* </ul>
*
* <p>Details on each of these are provided in their linked pages. </div> <a id="File_Naming"></a>
*
* <h3>File Naming</h3>
*
* <div>
*
* <p>All files belonging to a segment have the same name with varying extensions. The extensions
* correspond to the different file formats described below. When using the Compound File format
* (default for small segments) these files (except for the Segment info file, the Lock file, and
* Deleted documents file) are collapsed into a single .cfs file (see below for details)
*
* <p>Typically, all segments in an index are stored in a single directory, although this is not
* required.
*
* <p>File names are never re-used. That is, when any file is saved to the Directory it is given a
* never before used filename. This is achieved using a simple generations approach. For example,
* the first segments file is segments_1, then segments_2, etc. The generation is a sequential long
* integer represented in alpha-numeric (base 36) form. </div> <a id="file-names"></a>
*
* <h3>Summary of File Extensions</h3>
*
* <div>
*
* <p>The following table summarizes the names and extensions of the files in Lucene:
*
* <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
* <caption>lucene filenames by extension</caption>
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what documents are live</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}</td>
* <td>.dii, .dim</td>
* <td>Holds indexed points</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}</td>
* <td>.vec, .vem, .veq, vex</td>
* <td>Holds indexed vectors; <code>.vec</code> files contain the raw vector data,
* <code>.vem</code> the vector metadata, <code>.veq</code> the quantized vector data, and <code>.vex</code> the
* hnsw graph data.</td>
* </tr>
* </table>
*
* </div> <a id="Lock_File"></a>
*
* <h3>Lock File</h3>
*
* The write lock, which is stored in the index directory by default, is named "write.lock". If the
* lock directory is different from the index directory then the write lock will be named
* "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index
* directory. When this file is present, a writer is currently modifying the index (adding or
* removing documents). This lock file ensures that only one writer is modifying the index at a
* time. <a id="History"></a>
*
* <h3>History</h3>
*
* <p>Compatibility notes are provided in this document, describing how file formats have changed
* from prior versions:
*
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit
* lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching
* or adding/deleting of docs. When the new segments file is saved (committed), it will be
* written in the new file format (meaning no specific "upgrade" process is needed). But note
* that once a commit has occurred, pre-2.1 Lucene will not be able to read the index.
* <li>In version 2.3, the file format was changed to allow segments to share a single set of doc
* store (vectors &amp; stored fields) files. This allows for faster indexing in certain
* cases. The change is fully backwards compatible (in the same way as the lock-less commits
* change in 2.1).
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified
* UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">LUCENE-510</a> for
* details.
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData may be passed to
* IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N
* file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">LUCENE-1382</a> for
* details. Also, diagnostics were added to each segment written recording details about why
* it was written (due to flush, merge; which OS/JRE was used; etc.). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.
* <li>In version 3.0, compressed fields are no longer written to the index (they can still be
* read, but on merge the new segment will write them, uncompressed). See issue <a
* href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> for details.
* <li>In version 3.1, segments records the code version that created them. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors. See <a
* href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> for details.
* <li>In version 3.2, numeric fields are written as natively to stored fields file, previously
* they were stored in text format only.
* <li>In version 3.4, fields can omit position data while still indexing term frequencies.
* <li>In version 4.0, the format of the inverted index became extensible via the {@link
* org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues})
* was introduced. Normalization factors need no longer be a single byte, they can be any
* {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be
* unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into
* the postings lists. Payloads can be stored in the term vectors.
* <li>In version 4.1, the format of the postings list changed to use either of FOR compression or
* variable-byte encoding, depending upon the frequency of the term. Terms appearing only once
* were changed to inline directly into the term dictionary. Stored fields are compressed by
* default.
* <li>In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued
* type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields.
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.
* <li>In version 4.8, checksum footers were added to the end of each index file for improved data
* integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32
* checksum of the file.
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is
* suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk: addresses
* for binary fields and ord indexes for multi-valued fields.
* <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
* <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index
* sorting.
* <li>In version 7.0, DocValues have been improved to better support sparse doc values thanks to
* an iterator API.
* <li>In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term
* freq, normalization factor) pairs that may trigger the maximum score of the block. This
* information is recorded alongside skip data in order to be able to skip blocks of doc ids
* if they may not produce high enough scores. Additionally doc values and norms has been
* extended with jump-tables to make access O(1) instead of O(n), where n is the number of
* elements to skip when advancing in the data.
* <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
* performant encoding that is vectorized.
* <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to allow
* user-defined sorts to be used
* <li>In version 8.7, stored fields compression became adaptive to better handle documents with
* smaller stored fields.
* <li>In version 9.0, vector-valued fields were added.
* <li>In version 9.1, vector-valued fields were modified to add a graph hierarchy.
* <li>In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by
* IndexDISI. ordToDoc mappings was added to .vem.
* <li>In version 9.5, HNSW graph connections were changed to be delta-encoded with vints.
* Additionally, metadata file size improvements were made by delta-encoding nodes by graph
* layer and not writing the node ids for the zeroth layer.
* <li>In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector
* format to utilize int8 quantized vectors for float32 vector search.
* </ul>
*
* <a id="Limitations"></a>
*
* <h3>Limitations</h3>
*
* <div>
*
* <p>Lucene uses a Java <code>int</code> to refer to document numbers, and the index file format
* uses an <code>Int32</code> on-disk to store document numbers. This is a limitation of both the
* index file format and the current implementation. Eventually these should be replaced with either
* <code>UInt64</code> values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt
* VInt} values which have no limit. </div>
*/
/** Lucene 9.9 file format. */
package org.apache.lucene.codecs.lucene99;

View File

@ -3302,17 +3302,17 @@ public final class CheckIndex implements Closeable {
if (skipper.maxDocID(0) == NO_MORE_DOCS) {
break;
}
if (skipper.minDocID(0) < doc) {
throw new CheckIndexException(
"skipper dv iterator for field: "
+ fieldName
+ " reports wrong minDocID, got "
+ skipper.minDocID(0)
+ " < "
+ doc);
}
int levels = skipper.numLevels();
for (int level = 0; level < levels; level++) {
if (skipper.minDocID(level) < doc) {
throw new CheckIndexException(
"skipper dv iterator for field: "
+ fieldName
+ " reports wrong minDocID, got "
+ skipper.minDocID(level)
+ " < "
+ doc);
}
if (skipper.minDocID(level) > skipper.maxDocID(level)) {
throw new CheckIndexException(
"skipper dv iterator for field: "

View File

@ -430,10 +430,16 @@ final class DocumentsWriter implements Closeable, Accountable {
}
flushingDWPT = flushControl.doAfterDocument(dwpt);
} finally {
if (dwpt.isFlushPending() || dwpt.isAborted()) {
dwpt.unlock();
} else {
perThreadPool.marksAsFreeAndUnlock(dwpt);
// If a flush is occurring, we don't want to allow this dwpt to be reused
// If it is aborted, we shouldn't allow it to be reused
// If the deleteQueue is advanced, this means the maximum seqNo has been set and it cannot be
// reused
synchronized (flushControl) {
if (dwpt.isFlushPending() || dwpt.isAborted() || dwpt.isQueueAdvanced()) {
dwpt.unlock();
} else {
perThreadPool.marksAsFreeAndUnlock(dwpt);
}
}
assert dwpt.isHeldByCurrentThread() == false : "we didn't release the dwpt even on abort";
}

View File

@ -636,7 +636,7 @@ final class DocumentsWriterDeleteQueue implements Accountable, Closeable {
}
/** Returns <code>true</code> if it was advanced. */
boolean isAdvanced() {
synchronized boolean isAdvanced() {
return advanced;
}
}

View File

@ -718,6 +718,10 @@ final class DocumentsWriterPerThread implements Accountable, Lock {
return flushPending.get() == Boolean.TRUE;
}
boolean isQueueAdvanced() {
return deleteQueue.isAdvanced();
}
/** Sets this DWPT as flush pending. This can only be set once. */
void setFlushPending() {
flushPending.set(Boolean.TRUE);

View File

@ -138,6 +138,15 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
void marksAsFreeAndUnlock(DocumentsWriterPerThread state) {
final long ramBytesUsed = state.ramBytesUsed();
assert state.isFlushPending() == false
&& state.isAborted() == false
&& state.isQueueAdvanced() == false
: "DWPT has pending flush: "
+ state.isFlushPending()
+ " aborted="
+ state.isAborted()
+ " queueAdvanced="
+ state.isQueueAdvanced();
assert contains(state)
: "we tried to add a DWPT back to the pool but the pool doesn't know about this DWPT";
freeList.addAndUnlock(state, ramBytesUsed);

View File

@ -90,6 +90,12 @@ public abstract class LogMergePolicy extends MergePolicy {
/** If true, we pro-rate a segment's size by the percentage of non-deleted documents. */
protected boolean calibrateSizeByDeletes = true;
/**
* Target search concurrency. This merge policy will avoid creating segments that have more than
* {@code maxDoc / targetSearchConcurrency} documents.
*/
protected int targetSearchConcurrency = 1;
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
public LogMergePolicy() {
super(DEFAULT_NO_CFS_RATIO, MergePolicy.DEFAULT_MAX_CFS_SEGMENT_SIZE);
@ -131,6 +137,28 @@ public abstract class LogMergePolicy extends MergePolicy {
return calibrateSizeByDeletes;
}
/**
* Sets the target search concurrency. This prevents creating segments that are bigger than
* maxDoc/targetSearchConcurrency, which in turn makes the work parallelizable into
* targetSearchConcurrency slices of similar doc counts.
*
* <p><b>NOTE:</b> Configuring a value greater than 1 will increase the number of segments in the
* index linearly with the value of {@code targetSearchConcurrency} and also increase write
* amplification.
*/
public void setTargetSearchConcurrency(int targetSearchConcurrency) {
if (targetSearchConcurrency < 1) {
throw new IllegalArgumentException(
"targetSearchConcurrency must be >= 1 (got " + targetSearchConcurrency + ")");
}
this.targetSearchConcurrency = targetSearchConcurrency;
}
/** Returns the target search concurrency. */
public int getTargetSearchConcurrency() {
return targetSearchConcurrency;
}
/**
* Return the number of documents in the provided {@link SegmentCommitInfo}, pro-rated by
* percentage of non-deleted documents if {@link #setCalibrateSizeByDeletes} is set.
@ -484,8 +512,10 @@ public abstract class LogMergePolicy extends MergePolicy {
final Set<SegmentCommitInfo> mergingSegments = mergeContext.getMergingSegments();
int totalDocCount = 0;
for (int i = 0; i < numSegments; i++) {
final SegmentCommitInfo info = infos.info(i);
totalDocCount += sizeDocs(info, mergeContext);
long size = size(info, mergeContext);
// Floor tiny segments
@ -575,6 +605,9 @@ public abstract class LogMergePolicy extends MergePolicy {
mergeContext);
}
final int maxMergeDocs =
Math.min(this.maxMergeDocs, Math.ceilDiv(totalDocCount, targetSearchConcurrency));
// Finally, record all merges that are viable at this level:
int end = start + mergeFactor;
while (end <= 1 + upto) {

View File

@ -141,6 +141,9 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
bits.set(0, reader.maxDoc());
}
int numSoftDeletes = PendingSoftDeletes.applySoftDeletes(iterator, bits);
if (numSoftDeletes == 0) {
return reader;
}
int numDeletes = reader.numDeletedDocs() + numSoftDeletes;
int numDocs = reader.maxDoc() - numDeletes;
assert assertDocCounts(numDocs, numSoftDeletes, reader);

View File

@ -93,6 +93,7 @@ public class TieredMergePolicy extends MergePolicy {
private double segsPerTier = 10.0;
private double forceMergeDeletesPctAllowed = 10.0;
private double deletesPctAllowed = 20.0;
private int targetSearchConcurrency = 1;
/** Sole constructor, setting all settings to their defaults. */
public TieredMergePolicy() {
@ -257,6 +258,26 @@ public class TieredMergePolicy extends MergePolicy {
return segsPerTier;
}
/**
* Sets the target search concurrency. This prevents creating segments that are bigger than
* maxDoc/targetSearchConcurrency, which in turn makes the work parallelizable into
* targetSearchConcurrency slices of similar doc counts. It also makes merging less aggressive, as
* higher values result in indices that do less merging and have more segments
*/
public TieredMergePolicy setTargetSearchConcurrency(int targetSearchConcurrency) {
if (targetSearchConcurrency < 1) {
throw new IllegalArgumentException(
"targetSearchConcurrency must be >= 1 (got " + targetSearchConcurrency + ")");
}
this.targetSearchConcurrency = targetSearchConcurrency;
return this;
}
/** Returns the target search concurrency. */
public int getTargetSearchConcurrency() {
return targetSearchConcurrency;
}
private static class SegmentSizeAndDocs {
private final SegmentCommitInfo segInfo;
/// Size of the segment in bytes, pro-rated by the number of live documents.
@ -371,31 +392,40 @@ public class TieredMergePolicy extends MergePolicy {
// If we have too-large segments, grace them out of the maximum segment count
// If we're above certain thresholds of deleted docs, we can merge very large segments.
int tooBigCount = 0;
// We relax merging for the bigger segments for concurrency reasons, as we want to have several
// segments on the highest tier without over-merging on the lower tiers.
int concurrencyCount = 0;
iter = sortedInfos.iterator();
double allowedSegCount = 0;
// remove large segments from consideration under two conditions.
// 1> Overall percent deleted docs relatively small and this segment is larger than 50%
// maxSegSize
// 2> overall percent deleted docs large and this segment is large and has few deleted docs
while (iter.hasNext()) {
SegmentSizeAndDocs segSizeDocs = iter.next();
double segDelPct = 100 * (double) segSizeDocs.delCount / (double) segSizeDocs.maxDoc;
if (segSizeDocs.sizeInBytes > maxMergedSegmentBytes / 2
&& (totalDelPct <= deletesPctAllowed || segDelPct <= deletesPctAllowed)) {
iter.remove();
tooBigCount++; // Just for reporting purposes.
tooBigCount++;
totIndexBytes -= segSizeDocs.sizeInBytes;
allowedDelCount -= segSizeDocs.delCount;
} else if (concurrencyCount + tooBigCount < targetSearchConcurrency - 1) {
// Make sure we count a whole segment for the first targetSearchConcurrency-1 segments to
// avoid over merging on the lower levels.
concurrencyCount++;
allowedSegCount++;
totIndexBytes -= segSizeDocs.sizeInBytes;
}
}
allowedDelCount = Math.max(0, allowedDelCount);
final int mergeFactor = (int) Math.min(maxMergeAtOnce, segsPerTier);
// Compute max allowed segments in the index
// Compute max allowed segments for the remainder of the index
long levelSize = Math.max(minSegmentBytes, floorSegmentBytes);
long bytesLeft = totIndexBytes;
double allowedSegCount = 0;
while (true) {
final double segCountLevel = bytesLeft / (double) levelSize;
if (segCountLevel < segsPerTier || levelSize == maxMergedSegmentBytes) {
@ -408,7 +438,8 @@ public class TieredMergePolicy extends MergePolicy {
}
// allowedSegCount may occasionally be less than segsPerTier
// if segment sizes are below the floor size
allowedSegCount = Math.max(allowedSegCount, segsPerTier);
allowedSegCount = Math.max(allowedSegCount, Math.max(segsPerTier, targetSearchConcurrency));
int allowedDocCount = getMaxAllowedDocs(totalMaxDoc, totalDelDocs);
if (verbose(mergeContext) && tooBigCount > 0) {
message(
@ -419,7 +450,11 @@ public class TieredMergePolicy extends MergePolicy {
+ " (eligible count="
+ sortedInfos.size()
+ ") tooBigCount= "
+ tooBigCount,
+ tooBigCount
+ " allowedDocCount="
+ allowedDocCount
+ " vs doc count="
+ infos.totalMaxDoc(),
mergeContext);
}
return doFindMerges(
@ -428,6 +463,7 @@ public class TieredMergePolicy extends MergePolicy {
mergeFactor,
(int) allowedSegCount,
allowedDelCount,
allowedDocCount,
MERGE_TYPE.NATURAL,
mergeContext,
mergingBytes >= maxMergedSegmentBytes);
@ -439,6 +475,7 @@ public class TieredMergePolicy extends MergePolicy {
final int mergeFactor,
final int allowedSegCount,
final int allowedDelCount,
final int allowedDocCount,
final MERGE_TYPE mergeType,
MergeContext mergeContext,
boolean maxMergeIsRunning)
@ -522,16 +559,23 @@ public class TieredMergePolicy extends MergePolicy {
final List<SegmentCommitInfo> candidate = new ArrayList<>();
boolean hitTooLarge = false;
long bytesThisMerge = 0;
long docCountThisMerge = 0;
for (int idx = startIdx;
idx < sortedEligible.size()
&& candidate.size() < mergeFactor
&& bytesThisMerge < maxMergedSegmentBytes;
&& bytesThisMerge < maxMergedSegmentBytes
&& (bytesThisMerge < floorSegmentBytes || docCountThisMerge <= allowedDocCount);
idx++) {
final SegmentSizeAndDocs segSizeDocs = sortedEligible.get(idx);
final long segBytes = segSizeDocs.sizeInBytes;
if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes) {
hitTooLarge = true;
int segDocCount = segSizeDocs.maxDoc - segSizeDocs.delCount;
if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes
|| (totAfterMergeBytes > floorSegmentBytes
&& docCountThisMerge + segDocCount > allowedDocCount)) {
// Only set hitTooLarge when reaching the maximum byte size, as this will create
// segments of the maximum size which will no longer be eligible for merging for a long
// time (until they accumulate enough deletes).
hitTooLarge |= totAfterMergeBytes + segBytes > maxMergedSegmentBytes;
if (candidate.size() == 0) {
// We should never have something coming in that _cannot_ be merged, so handle
// singleton merges
@ -548,6 +592,7 @@ public class TieredMergePolicy extends MergePolicy {
}
candidate.add(segSizeDocs.segInfo);
bytesThisMerge += segBytes;
docCountThisMerge += segDocCount;
totAfterMergeBytes += segBytes;
}
@ -916,14 +961,13 @@ public class TieredMergePolicy extends MergePolicy {
final Set<SegmentCommitInfo> merging = mergeContext.getMergingSegments();
boolean haveWork = false;
int totalDelCount = 0;
for (SegmentCommitInfo info : infos) {
int delCount = mergeContext.numDeletesToMerge(info);
assert assertDelCount(delCount, info);
totalDelCount += delCount;
double pctDeletes = 100. * ((double) delCount) / info.info.maxDoc();
if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) {
haveWork = true;
break;
}
haveWork = haveWork || (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info));
}
if (haveWork == false) {
@ -950,11 +994,16 @@ public class TieredMergePolicy extends MergePolicy {
Integer.MAX_VALUE,
Integer.MAX_VALUE,
0,
getMaxAllowedDocs(infos.totalMaxDoc(), totalDelCount),
MERGE_TYPE.FORCE_MERGE_DELETES,
mergeContext,
false);
}
int getMaxAllowedDocs(int totalMaxDoc, int totalDelDocs) {
return Math.ceilDiv(totalMaxDoc - totalDelDocs, targetSearchConcurrency);
}
private long floorSize(long bytes) {
return Math.max(floorSegmentBytes, bytes);
}
@ -969,7 +1018,8 @@ public class TieredMergePolicy extends MergePolicy {
sb.append("segmentsPerTier=").append(segsPerTier).append(", ");
sb.append("maxCFSSegmentSizeMB=").append(getMaxCFSSegmentSizeMB()).append(", ");
sb.append("noCFSRatio=").append(noCFSRatio).append(", ");
sb.append("deletesPctAllowed=").append(deletesPctAllowed);
sb.append("deletesPctAllowed=").append(deletesPctAllowed).append(", ");
sb.append("targetSearchConcurrency=").append(targetSearchConcurrency);
return sb.toString();
}
}

View File

@ -66,10 +66,7 @@ public enum VectorSimilarityFunction {
* vectors to unit length, and instead use {@link VectorSimilarityFunction#DOT_PRODUCT}. You
* should only use this function if you need to preserve the original vectors and cannot normalize
* them in advance. The similarity score is normalised to assure it is positive.
*
* @deprecated Use MAXIMUM_INNER_PRODUCT or DOT_PRODUCT instead
*/
@Deprecated
COSINE {
@Override
public float compare(float[] v1, float[] v2) {

View File

@ -0,0 +1,834 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.internal.hppc;
import static org.apache.lucene.internal.hppc.HashContainers.*;
import java.util.Arrays;
import java.util.Iterator;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
/**
* A hash map of <code>int</code> to <code>long</code>, implemented using open addressing with
* linear probing for collision resolution.
*
* <p>Mostly forked and trimmed from com.carrotsearch.hppc.IntLongHashMap
*
* <p>github: https://github.com/carrotsearch/hppc release 0.10.0
*
* @lucene.internal
*/
public class IntLongHashMap
implements Iterable<IntLongHashMap.IntLongCursor>, Accountable, Cloneable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(IntLongHashMap.class);
/** The array holding keys. */
public int[] keys;
/** The array holding values. */
public long[] values;
/**
* The number of stored keys (assigned key slots), excluding the special "empty" key, if any (use
* {@link #size()} instead).
*
* @see #size()
*/
protected int assigned;
/** Mask for slot scans in {@link #keys}. */
protected int mask;
/** Expand (rehash) {@link #keys} when {@link #assigned} hits this value. */
protected int resizeAt;
/** Special treatment for the "empty slot" key marker. */
protected boolean hasEmptyKey;
/** The load factor for {@link #keys}. */
protected double loadFactor;
/** Seed used to ensure the hash iteration order is different from an iteration to another. */
protected int iterationSeed;
/** New instance with sane defaults. */
public IntLongHashMap() {
this(DEFAULT_EXPECTED_ELEMENTS);
}
/**
* New instance with sane defaults.
*
* @param expectedElements The expected number of elements guaranteed not to cause buffer
* expansion (inclusive).
*/
public IntLongHashMap(int expectedElements) {
this(expectedElements, DEFAULT_LOAD_FACTOR);
}
/**
* New instance with the provided defaults.
*
* @param expectedElements The expected number of elements guaranteed not to cause a rehash
* (inclusive).
* @param loadFactor The load factor for internal buffers. Insane load factors (zero, full
* capacity) are rejected by {@link #verifyLoadFactor(double)}.
*/
public IntLongHashMap(int expectedElements, double loadFactor) {
this.loadFactor = verifyLoadFactor(loadFactor);
iterationSeed = ITERATION_SEED.incrementAndGet();
ensureCapacity(expectedElements);
}
/** Create a hash map from all key-value pairs of another container. */
public IntLongHashMap(IntLongHashMap container) {
this(container.size());
putAll(container);
}
public long put(int key, long value) {
assert assigned < mask + 1;
final int mask = this.mask;
if (((key) == 0)) {
long previousValue = hasEmptyKey ? values[mask + 1] : 0L;
hasEmptyKey = true;
values[mask + 1] = value;
return previousValue;
} else {
final int[] keys = this.keys;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
final long previousValue = values[slot];
values[slot] = value;
return previousValue;
}
slot = (slot + 1) & mask;
}
if (assigned == resizeAt) {
allocateThenInsertThenRehash(slot, key, value);
} else {
keys[slot] = key;
values[slot] = value;
}
assigned++;
return 0L;
}
}
/**
* If the specified key is not already associated with a value, associates it with the given
* value.
*
* @return {@code true} if {@code key} did not exist and {@code value} was placed in the map,
* {@code false} otherwise.
*/
public boolean putIfAbsent(int key, long value) {
int keyIndex = indexOf(key);
if (indexExists(keyIndex)) {
return false;
} else {
indexInsert(keyIndex, key, value);
return true;
}
}
/** Puts all key/value pairs from a given iterable into this map. */
public int putAll(Iterable<? extends IntLongCursor> iterable) {
final int count = size();
for (IntLongCursor c : iterable) {
put(c.key, c.value);
}
return size() - count;
}
/**
* If <code>key</code> exists, <code>putValue</code> is inserted into the map, otherwise any
* existing value is incremented by <code>additionValue</code>.
*
* @param key The key of the value to adjust.
* @param putValue The value to put if <code>key</code> does not exist.
* @param incrementValue The value to add to the existing value if <code>key</code> exists.
* @return Returns the current value associated with <code>key</code> (after changes).
*/
public long putOrAdd(int key, long putValue, long incrementValue) {
assert assigned < mask + 1;
int keyIndex = indexOf(key);
if (indexExists(keyIndex)) {
putValue = values[keyIndex] + incrementValue;
indexReplace(keyIndex, putValue);
} else {
indexInsert(keyIndex, key, putValue);
}
return putValue;
}
/**
* Adds <code>incrementValue</code> to any existing value for the given <code>key</code> or
* inserts <code>incrementValue</code> if <code>key</code> did not previously exist.
*
* @param key The key of the value to adjust.
* @param incrementValue The value to put or add to the existing value if <code>key</code> exists.
* @return Returns the current value associated with <code>key</code> (after changes).
*/
public long addTo(int key, long incrementValue) {
return putOrAdd(key, incrementValue, incrementValue);
}
/**
* Remove all values at the given key. The default value for the key type is returned if the value
* does not exist in the map.
*/
public long remove(int key) {
final int mask = this.mask;
if (((key) == 0)) {
if (!hasEmptyKey) {
return 0L;
}
hasEmptyKey = false;
long previousValue = values[mask + 1];
values[mask + 1] = 0L;
return previousValue;
} else {
final int[] keys = this.keys;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
final long previousValue = values[slot];
shiftConflictingKeys(slot);
return previousValue;
}
slot = (slot + 1) & mask;
}
return 0L;
}
}
public long get(int key) {
if (((key) == 0)) {
return hasEmptyKey ? values[mask + 1] : 0L;
} else {
final int[] keys = this.keys;
final int mask = this.mask;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
return values[slot];
}
slot = (slot + 1) & mask;
}
return 0L;
}
}
public long getOrDefault(int key, long defaultValue) {
if (((key) == 0)) {
return hasEmptyKey ? values[mask + 1] : defaultValue;
} else {
final int[] keys = this.keys;
final int mask = this.mask;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
return values[slot];
}
slot = (slot + 1) & mask;
}
return defaultValue;
}
}
public boolean containsKey(int key) {
if (((key) == 0)) {
return hasEmptyKey;
} else {
final int[] keys = this.keys;
final int mask = this.mask;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
return true;
}
slot = (slot + 1) & mask;
}
return false;
}
}
public int indexOf(int key) {
final int mask = this.mask;
if (((key) == 0)) {
return hasEmptyKey ? mask + 1 : ~(mask + 1);
} else {
final int[] keys = this.keys;
int slot = hashKey(key) & mask;
int existing;
while (!((existing = keys[slot]) == 0)) {
if (((key) == (existing))) {
return slot;
}
slot = (slot + 1) & mask;
}
return ~slot;
}
}
public boolean indexExists(int index) {
assert index < 0 || (index >= 0 && index <= mask) || (index == mask + 1 && hasEmptyKey);
return index >= 0;
}
public long indexGet(int index) {
assert index >= 0 : "The index must point at an existing key.";
assert index <= mask || (index == mask + 1 && hasEmptyKey);
return values[index];
}
public long indexReplace(int index, long newValue) {
assert index >= 0 : "The index must point at an existing key.";
assert index <= mask || (index == mask + 1 && hasEmptyKey);
long previousValue = values[index];
values[index] = newValue;
return previousValue;
}
public void indexInsert(int index, int key, long value) {
assert index < 0 : "The index must not point at an existing key.";
index = ~index;
if (((key) == 0)) {
assert index == mask + 1;
values[index] = value;
hasEmptyKey = true;
} else {
assert ((keys[index]) == 0);
if (assigned == resizeAt) {
allocateThenInsertThenRehash(index, key, value);
} else {
keys[index] = key;
values[index] = value;
}
assigned++;
}
}
public long indexRemove(int index) {
assert index >= 0 : "The index must point at an existing key.";
assert index <= mask || (index == mask + 1 && hasEmptyKey);
long previousValue = values[index];
if (index > mask) {
assert index == mask + 1;
hasEmptyKey = false;
values[index] = 0L;
} else {
shiftConflictingKeys(index);
}
return previousValue;
}
public void clear() {
assigned = 0;
hasEmptyKey = false;
Arrays.fill(keys, 0);
}
public void release() {
assigned = 0;
hasEmptyKey = false;
keys = null;
values = null;
ensureCapacity(DEFAULT_EXPECTED_ELEMENTS);
}
public int size() {
return assigned + (hasEmptyKey ? 1 : 0);
}
public boolean isEmpty() {
return size() == 0;
}
@Override
public int hashCode() {
int h = hasEmptyKey ? 0xDEADBEEF : 0;
for (IntLongCursor c : this) {
h += BitMixer.mix(c.key) + BitMixer.mix(c.value);
}
return h;
}
@Override
public boolean equals(Object obj) {
return (this == obj)
|| (obj != null && getClass() == obj.getClass() && equalElements(getClass().cast(obj)));
}
/** Return true if all keys of some other container exist in this container. */
protected boolean equalElements(IntLongHashMap other) {
if (other.size() != size()) {
return false;
}
for (IntLongCursor c : other) {
int key = c.key;
if (!containsKey(key) || !((c.value) == (get(key)))) {
return false;
}
}
return true;
}
/**
* Ensure this container can hold at least the given number of keys (entries) without resizing its
* buffers.
*
* @param expectedElements The total number of keys, inclusive.
*/
public void ensureCapacity(int expectedElements) {
if (expectedElements > resizeAt || keys == null) {
final int[] prevKeys = this.keys;
final long[] prevValues = this.values;
allocateBuffers(minBufferSize(expectedElements, loadFactor));
if (prevKeys != null && !isEmpty()) {
rehash(prevKeys, prevValues);
}
}
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(keys) + RamUsageEstimator.sizeOf(values);
}
/**
* Provides the next iteration seed used to build the iteration starting slot and offset
* increment. This method does not need to be synchronized, what matters is that each thread gets
* a sequence of varying seeds.
*/
protected int nextIterationSeed() {
return iterationSeed = BitMixer.mixPhi(iterationSeed);
}
/** An iterator implementation for {@link #iterator}. */
private final class EntryIterator extends AbstractIterator<IntLongCursor> {
private final IntLongCursor cursor;
private final int increment;
private int index;
private int slot;
public EntryIterator() {
cursor = new IntLongCursor();
int seed = nextIterationSeed();
increment = iterationIncrement(seed);
slot = seed & mask;
}
@Override
protected IntLongCursor fetch() {
final int mask = IntLongHashMap.this.mask;
while (index <= mask) {
int existing;
index++;
slot = (slot + increment) & mask;
if (!((existing = keys[slot]) == 0)) {
cursor.index = slot;
cursor.key = existing;
cursor.value = values[slot];
return cursor;
}
}
if (index == mask + 1 && hasEmptyKey) {
cursor.index = index;
cursor.key = 0;
cursor.value = values[index++];
return cursor;
}
return done();
}
}
@Override
public Iterator<IntLongCursor> iterator() {
return new EntryIterator();
}
/** Returns a specialized view of the keys of this associated container. */
public KeysContainer keys() {
return new KeysContainer();
}
/** A view of the keys inside this hash map. */
public final class KeysContainer implements Iterable<IntCursor> {
@Override
public Iterator<IntCursor> iterator() {
return new KeysIterator();
}
public int size() {
return IntLongHashMap.this.size();
}
public int[] toArray() {
int[] array = new int[size()];
int i = 0;
for (IntCursor cursor : this) {
array[i++] = cursor.value;
}
return array;
}
}
/** An iterator over the set of assigned keys. */
private final class KeysIterator extends AbstractIterator<IntCursor> {
private final IntCursor cursor;
private final int increment;
private int index;
private int slot;
public KeysIterator() {
cursor = new IntCursor();
int seed = nextIterationSeed();
increment = iterationIncrement(seed);
slot = seed & mask;
}
@Override
protected IntCursor fetch() {
final int mask = IntLongHashMap.this.mask;
while (index <= mask) {
int existing;
index++;
slot = (slot + increment) & mask;
if (!((existing = keys[slot]) == 0)) {
cursor.index = slot;
cursor.value = existing;
return cursor;
}
}
if (index == mask + 1 && hasEmptyKey) {
cursor.index = index++;
cursor.value = 0;
return cursor;
}
return done();
}
}
/**
* @return Returns a container with all values stored in this map.
*/
public ValuesContainer values() {
return new ValuesContainer();
}
/** A view over the set of values of this map. */
public final class ValuesContainer implements Iterable<LongCursor> {
@Override
public Iterator<LongCursor> iterator() {
return new ValuesIterator();
}
public long[] toArray() {
long[] array = new long[size()];
int i = 0;
for (LongCursor cursor : this) {
array[i++] = cursor.value;
}
return array;
}
}
/** An iterator over the set of assigned values. */
private final class ValuesIterator extends AbstractIterator<LongCursor> {
private final LongCursor cursor;
private final int increment;
private int index;
private int slot;
public ValuesIterator() {
cursor = new LongCursor();
int seed = nextIterationSeed();
increment = iterationIncrement(seed);
slot = seed & mask;
}
@Override
protected LongCursor fetch() {
final int mask = IntLongHashMap.this.mask;
while (index <= mask) {
index++;
slot = (slot + increment) & mask;
if (!((keys[slot]) == 0)) {
cursor.index = slot;
cursor.value = values[slot];
return cursor;
}
}
if (index == mask + 1 && hasEmptyKey) {
cursor.index = index;
cursor.value = values[index++];
return cursor;
}
return done();
}
}
@Override
public IntLongHashMap clone() {
try {
IntLongHashMap cloned = (IntLongHashMap) super.clone();
cloned.keys = keys.clone();
cloned.values = values.clone();
cloned.hasEmptyKey = hasEmptyKey;
cloned.iterationSeed = ITERATION_SEED.incrementAndGet();
return cloned;
} catch (CloneNotSupportedException e) {
throw new RuntimeException(e);
}
}
/** Convert the contents of this map to a human-friendly string. */
@Override
public String toString() {
final StringBuilder buffer = new StringBuilder();
buffer.append("[");
boolean first = true;
for (IntLongCursor cursor : this) {
if (!first) {
buffer.append(", ");
}
buffer.append(cursor.key);
buffer.append("=>");
buffer.append(cursor.value);
first = false;
}
buffer.append("]");
return buffer.toString();
}
/** Creates a hash map from two index-aligned arrays of key-value pairs. */
public static IntLongHashMap from(int[] keys, long[] values) {
if (keys.length != values.length) {
throw new IllegalArgumentException(
"Arrays of keys and values must have an identical length.");
}
IntLongHashMap map = new IntLongHashMap(keys.length);
for (int i = 0; i < keys.length; i++) {
map.put(keys[i], values[i]);
}
return map;
}
/**
* Returns a hash code for the given key.
*
* <p>The output from this function should evenly distribute keys across the entire integer range.
*/
protected int hashKey(int key) {
assert !((key) == 0); // Handled as a special case (empty slot marker).
return BitMixer.mixPhi(key);
}
/**
* Validate load factor range and return it. Override and suppress if you need insane load
* factors.
*/
protected double verifyLoadFactor(double loadFactor) {
checkLoadFactor(loadFactor, MIN_LOAD_FACTOR, MAX_LOAD_FACTOR);
return loadFactor;
}
/** Rehash from old buffers to new buffers. */
protected void rehash(int[] fromKeys, long[] fromValues) {
assert fromKeys.length == fromValues.length
&& HashContainers.checkPowerOfTwo(fromKeys.length - 1);
// Rehash all stored key/value pairs into the new buffers.
final int[] keys = this.keys;
final long[] values = this.values;
final int mask = this.mask;
int existing;
// Copy the zero element's slot, then rehash everything else.
int from = fromKeys.length - 1;
keys[keys.length - 1] = fromKeys[from];
values[values.length - 1] = fromValues[from];
while (--from >= 0) {
if (!((existing = fromKeys[from]) == 0)) {
int slot = hashKey(existing) & mask;
while (!((keys[slot]) == 0)) {
slot = (slot + 1) & mask;
}
keys[slot] = existing;
values[slot] = fromValues[from];
}
}
}
/**
* Allocate new internal buffers. This method attempts to allocate and assign internal buffers
* atomically (either allocations succeed or not).
*/
protected void allocateBuffers(int arraySize) {
assert Integer.bitCount(arraySize) == 1;
// Ensure no change is done if we hit an OOM.
int[] prevKeys = this.keys;
long[] prevValues = this.values;
try {
int emptyElementSlot = 1;
this.keys = (new int[arraySize + emptyElementSlot]);
this.values = (new long[arraySize + emptyElementSlot]);
} catch (OutOfMemoryError e) {
this.keys = prevKeys;
this.values = prevValues;
throw new BufferAllocationException(
"Not enough memory to allocate buffers for rehashing: %,d -> %,d",
e, this.mask + 1, arraySize);
}
this.resizeAt = expandAtCount(arraySize, loadFactor);
this.mask = arraySize - 1;
}
/**
* This method is invoked when there is a new key/ value pair to be inserted into the buffers but
* there is not enough empty slots to do so.
*
* <p>New buffers are allocated. If this succeeds, we know we can proceed with rehashing so we
* assign the pending element to the previous buffer (possibly violating the invariant of having
* at least one empty slot) and rehash all keys, substituting new buffers at the end.
*/
protected void allocateThenInsertThenRehash(int slot, int pendingKey, long pendingValue) {
assert assigned == resizeAt && ((keys[slot]) == 0) && !((pendingKey) == 0);
// Try to allocate new buffers first. If we OOM, we leave in a consistent state.
final int[] prevKeys = this.keys;
final long[] prevValues = this.values;
allocateBuffers(nextBufferSize(mask + 1, size(), loadFactor));
assert this.keys.length > prevKeys.length;
// We have succeeded at allocating new data so insert the pending key/value at
// the free slot in the old arrays before rehashing.
prevKeys[slot] = pendingKey;
prevValues[slot] = pendingValue;
// Rehash old keys, including the pending key.
rehash(prevKeys, prevValues);
}
/**
* Shift all the slot-conflicting keys and values allocated to (and including) <code>slot</code>.
*/
protected void shiftConflictingKeys(int gapSlot) {
final int[] keys = this.keys;
final long[] values = this.values;
final int mask = this.mask;
// Perform shifts of conflicting keys to fill in the gap.
int distance = 0;
while (true) {
final int slot = (gapSlot + (++distance)) & mask;
final int existing = keys[slot];
if (((existing) == 0)) {
break;
}
final int idealSlot = hashKey(existing);
final int shift = (slot - idealSlot) & mask;
if (shift >= distance) {
// Entry at this position was originally at or before the gap slot.
// Move the conflict-shifted entry to the gap's position and repeat the procedure
// for any entries to the right of the current position, treating it
// as the new gap.
keys[gapSlot] = existing;
values[gapSlot] = values[slot];
gapSlot = slot;
distance = 0;
}
}
// Mark the last found gap slot without a conflict as empty.
keys[gapSlot] = 0;
values[gapSlot] = 0L;
assigned--;
}
/** Forked from HPPC, holding int index,key and value */
public static final class IntLongCursor {
/**
* The current key and value's index in the container this cursor belongs to. The meaning of
* this index is defined by the container (usually it will be an index in the underlying storage
* buffer).
*/
public int index;
/** The current key. */
public int key;
/** The current value. */
public long value;
@Override
public String toString() {
return "[cursor, index: " + index + ", key: " + key + ", value: " + value + "]";
}
}
}

View File

@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.internal.vectorization;
import java.io.IOException;
import org.apache.lucene.store.IndexInput;
final class DefaultPostingDecodingUtil extends PostingDecodingUtil {
protected final IndexInput in;
public DefaultPostingDecodingUtil(IndexInput in) {
this.in = in;
}
@Override
public void splitLongs(
int count, long[] b, int bShift, long bMask, long[] c, int cIndex, long cMask)
throws IOException {
assert count <= 64;
in.readLongs(c, cIndex, count);
// The below loop is auto-vectorized
for (int i = 0; i < count; ++i) {
b[i] = (c[cIndex + i] >>> bShift) & bMask;
c[cIndex + i] &= cMask;
}
}
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.internal.vectorization;
import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer;
import org.apache.lucene.codecs.hnsw.FlatVectorsScorer;
import org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorScorer;
import org.apache.lucene.store.IndexInput;
/** Default provider returning scalar implementations. */
final class DefaultVectorizationProvider extends VectorizationProvider {
@ -44,4 +45,9 @@ final class DefaultVectorizationProvider extends VectorizationProvider {
public FlatVectorsScorer getLucene99ScalarQuantizedVectorsScorer() {
return new Lucene99ScalarQuantizedVectorScorer(getLucene99FlatVectorsScorer());
}
@Override
public PostingDecodingUtil newPostingDecodingUtil(IndexInput input) {
return new DefaultPostingDecodingUtil(input);
}
}

View File

@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.internal.vectorization;
import java.io.IOException;
/** Utility class to decode postings. */
public abstract class PostingDecodingUtil {
/**
* Read {@code count} longs. This number must not exceed 64. Apply shift {@code bShift} and mask
* {@code bMask} and store the result in {@code b} starting at offset 0. Apply mask {@code cMask}
* and store the result in {@code c} starting at offset {@code cIndex}.
*/
public abstract void splitLongs(
int count, long[] b, int bShift, long bMask, long[] c, int cIndex, long cMask)
throws IOException;
}

View File

@ -17,6 +17,7 @@
package org.apache.lucene.internal.vectorization;
import java.io.IOException;
import java.lang.StackWalker.StackFrame;
import java.lang.invoke.MethodHandles;
import java.lang.invoke.MethodType;
@ -28,6 +29,7 @@ import java.util.function.Predicate;
import java.util.logging.Logger;
import java.util.stream.Stream;
import org.apache.lucene.codecs.hnsw.FlatVectorsScorer;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.VectorUtil;
@ -98,6 +100,9 @@ public abstract class VectorizationProvider {
/** Returns a FlatVectorsScorer that supports scalar quantized vectors in the Lucene99 format. */
public abstract FlatVectorsScorer getLucene99ScalarQuantizedVectorsScorer();
/** Create a new {@link PostingDecodingUtil} for the given {@link IndexInput}. */
public abstract PostingDecodingUtil newPostingDecodingUtil(IndexInput input) throws IOException;
// *** Lookup mechanism: ***
private static final Logger LOG = Logger.getLogger(VectorizationProvider.class.getName());
@ -187,7 +192,8 @@ public abstract class VectorizationProvider {
private static final Set<String> VALID_CALLERS =
Set.of(
"org.apache.lucene.codecs.hnsw.FlatVectorScorerUtil",
"org.apache.lucene.util.VectorUtil");
"org.apache.lucene.util.VectorUtil",
"org.apache.lucene.codecs.lucene912.PostingIndexInput");
private static void ensureCaller() {
final boolean validCaller =

View File

@ -154,38 +154,17 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper<Q extends MultiTermQue
List<TermAndState> collectedTerms)
throws IOException;
private IOSupplier<WeightOrDocIdSetIterator> rewrite(LeafReaderContext context, Terms terms)
throws IOException {
assert terms != null;
final int fieldDocCount = terms.getDocCount();
final TermsEnum termsEnum = q.getTermsEnum(terms);
assert termsEnum != null;
final List<TermAndState> collectedTerms = new ArrayList<>();
boolean collectResult = collectTerms(fieldDocCount, termsEnum, collectedTerms);
if (collectResult && collectedTerms.isEmpty()) {
return null;
private WeightOrDocIdSetIterator rewriteAsBooleanQuery(
LeafReaderContext context, List<TermAndState> collectedTerms) throws IOException {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
for (TermAndState t : collectedTerms) {
final TermStates termStates = new TermStates(searcher.getTopReaderContext());
termStates.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
bq.add(new TermQuery(new Term(q.field, t.term), termStates), BooleanClause.Occur.SHOULD);
}
return () -> {
if (collectResult) {
// build a boolean query
BooleanQuery.Builder bq = new BooleanQuery.Builder();
for (TermAndState t : collectedTerms) {
final TermStates termStates = new TermStates(searcher.getTopReaderContext());
termStates.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
bq.add(
new TermQuery(new Term(q.field, t.term), termStates), BooleanClause.Occur.SHOULD);
}
Query q = new ConstantScoreQuery(bq.build());
final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score());
return new WeightOrDocIdSetIterator(weight);
} else {
// Too many terms to rewrite as a simple bq.
// Invoke rewriteInner logic to handle rewriting:
return rewriteInner(context, fieldDocCount, terms, termsEnum, collectedTerms);
}
};
Query q = new ConstantScoreQuery(bq.build());
final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score());
return new WeightOrDocIdSetIterator(weight);
}
private boolean collectTerms(int fieldDocCount, TermsEnum termsEnum, List<TermAndState> terms)
@ -240,9 +219,44 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper<Q extends MultiTermQue
return null;
}
final long cost = estimateCost(terms, q.getTermsCount());
IOSupplier<WeightOrDocIdSetIterator> weightOrIteratorSupplier = rewrite(context, terms);
if (weightOrIteratorSupplier == null) return null;
assert terms != null;
final int fieldDocCount = terms.getDocCount();
final TermsEnum termsEnum = q.getTermsEnum(terms);
assert termsEnum != null;
List<TermAndState> collectedTerms = new ArrayList<>();
boolean collectResult = collectTerms(fieldDocCount, termsEnum, collectedTerms);
final long cost;
if (collectResult) {
// Return a null supplier if no query terms were in the segment:
if (collectedTerms.isEmpty()) {
return null;
}
// TODO: Instead of replicating the cost logic of a BooleanQuery we could consider rewriting
// to a BQ eagerly at this point and delegating to its cost method (instead of lazily
// rewriting on #get). Not sure what the performance hit would be of doing this though.
long sumTermCost = 0;
for (TermAndState collectedTerm : collectedTerms) {
sumTermCost += collectedTerm.docFreq;
}
cost = sumTermCost;
} else {
cost = estimateCost(terms, q.getTermsCount());
}
IOSupplier<WeightOrDocIdSetIterator> weightOrIteratorSupplier =
() -> {
if (collectResult) {
return rewriteAsBooleanQuery(context, collectedTerms);
} else {
// Too many terms to rewrite as a simple bq.
// Invoke rewriteInner logic to handle rewriting:
return rewriteInner(context, fieldDocCount, terms, termsEnum, collectedTerms);
}
};
return new ScorerSupplier() {
@Override

View File

@ -23,6 +23,8 @@ import java.util.List;
import java.util.Objects;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.search.knn.KnnCollectorManager;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.Bits;
@ -58,10 +60,19 @@ abstract class AbstractVectorSimilarityQuery extends Query {
this.filter = filter;
}
protected KnnCollectorManager getKnnCollectorManager() {
return (visitedLimit, context) ->
new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitedLimit);
}
abstract VectorScorer createVectorScorer(LeafReaderContext context) throws IOException;
protected abstract TopDocs approximateSearch(
LeafReaderContext context, Bits acceptDocs, int visitLimit) throws IOException;
LeafReaderContext context,
Bits acceptDocs,
int visitLimit,
KnnCollectorManager knnCollectorManager)
throws IOException;
@Override
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
@ -72,6 +83,10 @@ abstract class AbstractVectorSimilarityQuery extends Query {
? null
: searcher.createWeight(searcher.rewrite(filter), ScoreMode.COMPLETE_NO_SCORES, 1);
final QueryTimeout queryTimeout = searcher.getTimeout();
final TimeLimitingKnnCollectorManager timeLimitingKnnCollectorManager =
new TimeLimitingKnnCollectorManager(getKnnCollectorManager(), queryTimeout);
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
if (filterWeight != null) {
@ -103,16 +118,14 @@ abstract class AbstractVectorSimilarityQuery extends Query {
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
LeafReader leafReader = context.reader();
Bits liveDocs = leafReader.getLiveDocs();
final Scorer vectorSimilarityScorer;
// If there is no filter
if (filterWeight == null) {
// Return exhaustive results
TopDocs results = approximateSearch(context, liveDocs, Integer.MAX_VALUE);
if (results.scoreDocs.length == 0) {
return null;
}
vectorSimilarityScorer =
VectorSimilarityScorer.fromScoreDocs(this, boost, results.scoreDocs);
TopDocs results =
approximateSearch(
context, liveDocs, Integer.MAX_VALUE, timeLimitingKnnCollectorManager);
return VectorSimilarityScorerSupplier.fromScoreDocs(boost, results.scoreDocs);
} else {
Scorer scorer = filterWeight.scorer(context);
if (scorer == null) {
@ -143,27 +156,23 @@ abstract class AbstractVectorSimilarityQuery extends Query {
}
// Perform an approximate search
TopDocs results = approximateSearch(context, acceptDocs, cardinality);
TopDocs results =
approximateSearch(context, acceptDocs, cardinality, timeLimitingKnnCollectorManager);
// If the limit was exhausted
if (results.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) {
// Return a lazy-loading iterator
vectorSimilarityScorer =
VectorSimilarityScorer.fromAcceptDocs(
this,
boost,
createVectorScorer(context),
new BitSetIterator(acceptDocs, cardinality),
resultSimilarity);
} else if (results.scoreDocs.length == 0) {
return null;
} else {
if (results.totalHits.relation == TotalHits.Relation.EQUAL_TO
// Return partial results only when timeout is met
|| (queryTimeout != null && queryTimeout.shouldExit())) {
// Return an iterator over the collected results
vectorSimilarityScorer =
VectorSimilarityScorer.fromScoreDocs(this, boost, results.scoreDocs);
return VectorSimilarityScorerSupplier.fromScoreDocs(boost, results.scoreDocs);
} else {
// Return a lazy-loading iterator
return VectorSimilarityScorerSupplier.fromAcceptDocs(
boost,
createVectorScorer(context),
new BitSetIterator(acceptDocs, cardinality),
resultSimilarity);
}
}
return new DefaultScorerSupplier(vectorSimilarityScorer);
}
@Override
@ -197,16 +206,20 @@ abstract class AbstractVectorSimilarityQuery extends Query {
return Objects.hash(field, traversalSimilarity, resultSimilarity, filter);
}
private static class VectorSimilarityScorer extends Scorer {
private static class VectorSimilarityScorerSupplier extends ScorerSupplier {
final DocIdSetIterator iterator;
final float[] cachedScore;
VectorSimilarityScorer(DocIdSetIterator iterator, float[] cachedScore) {
VectorSimilarityScorerSupplier(DocIdSetIterator iterator, float[] cachedScore) {
this.iterator = iterator;
this.cachedScore = cachedScore;
}
static VectorSimilarityScorer fromScoreDocs(Weight weight, float boost, ScoreDoc[] scoreDocs) {
static VectorSimilarityScorerSupplier fromScoreDocs(float boost, ScoreDoc[] scoreDocs) {
if (scoreDocs.length == 0) {
return null;
}
// Sort in ascending order of docid
Arrays.sort(scoreDocs, Comparator.comparingInt(scoreDoc -> scoreDoc.doc));
@ -252,18 +265,15 @@ abstract class AbstractVectorSimilarityQuery extends Query {
}
};
return new VectorSimilarityScorer(iterator, cachedScore);
return new VectorSimilarityScorerSupplier(iterator, cachedScore);
}
static VectorSimilarityScorer fromAcceptDocs(
Weight weight,
float boost,
VectorScorer scorer,
DocIdSetIterator acceptDocs,
float threshold) {
static VectorSimilarityScorerSupplier fromAcceptDocs(
float boost, VectorScorer scorer, DocIdSetIterator acceptDocs, float threshold) {
if (scorer == null) {
return null;
}
float[] cachedScore = new float[1];
DocIdSetIterator vectorIterator = scorer.iterator();
DocIdSetIterator conjunction =
@ -281,27 +291,37 @@ abstract class AbstractVectorSimilarityQuery extends Query {
}
};
return new VectorSimilarityScorer(iterator, cachedScore);
return new VectorSimilarityScorerSupplier(iterator, cachedScore);
}
@Override
public int docID() {
return iterator.docID();
public Scorer get(long leadCost) {
return new Scorer() {
@Override
public int docID() {
return iterator.docID();
}
@Override
public DocIdSetIterator iterator() {
return iterator;
}
@Override
public float getMaxScore(int upTo) {
return Float.POSITIVE_INFINITY;
}
@Override
public float score() {
return cachedScore[0];
}
};
}
@Override
public DocIdSetIterator iterator() {
return iterator;
}
@Override
public float getMaxScore(int upTo) {
return Float.POSITIVE_INFINITY;
}
@Override
public float score() {
return cachedScore[0];
public long cost() {
return iterator.cost();
}
}
}

View File

@ -30,7 +30,7 @@ import org.apache.lucene.util.PriorityQueue;
*/
final class BooleanScorer extends BulkScorer {
static final int SHIFT = 11;
static final int SHIFT = 12;
static final int SIZE = 1 << SHIFT;
static final int MASK = SIZE - 1;
static final int SET_SIZE = 1 << (SHIFT - 6);

View File

@ -23,6 +23,7 @@ import java.util.Objects;
import org.apache.lucene.document.KnnByteVectorField;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.knn.KnnCollectorManager;
import org.apache.lucene.util.Bits;
/**
@ -106,10 +107,13 @@ public class ByteVectorSimilarityQuery extends AbstractVectorSimilarityQuery {
@Override
@SuppressWarnings("resource")
protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitLimit)
protected TopDocs approximateSearch(
LeafReaderContext context,
Bits acceptDocs,
int visitLimit,
KnnCollectorManager knnCollectorManager)
throws IOException {
KnnCollector collector =
new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitLimit);
KnnCollector collector = knnCollectorManager.newCollector(visitLimit, context);
context.reader().searchNearestVectors(field, target, collector, acceptDocs);
return collector.topDocs();
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
/**
* This class wraps {@link CollectorManager} and owns the collectors the manager creates. It is
* convenient that clients of the class don't have to worry about keeping the list of collectors, as
* well as about making the collector's type (C) compatible when reduce is called. Instances of this
* class cache results of {@link CollectorManager#reduce(Collection)}.
*
* <p>Note that instance of this class ignores any {@link Collector} created by {@link
* CollectorManager#newCollector()} directly, not through {@link #newCollector()}
*
* @lucene.experimental
*/
public final class CollectorOwner<C extends Collector, T> {
private final CollectorManager<C, T> manager;
private T result;
private boolean reduced;
// TODO: For IndexSearcher, the list doesn't have to be synchronized
// because we create new collectors sequentially. Drill sideways creates new collectors in
// DrillSidewaysQuery#Weight#bulkScorer which is already called concurrently.
// I think making the list synchronized here is not a huge concern, at the same time, do we want
// to do something about it?
// e.g. have boolean property in constructor that makes it threads friendly when set?
private final List<C> collectors = Collections.synchronizedList(new ArrayList<>());
public CollectorOwner(CollectorManager<C, T> manager) {
this.manager = manager;
}
/** Return a new {@link Collector}. This must return a different instance on each call. */
public C newCollector() throws IOException {
C collector = manager.newCollector();
collectors.add(collector);
return collector;
}
public C getCollector(int i) {
return collectors.get(i);
}
/**
* Returns result of {@link CollectorManager#reduce(Collection)}. The result is cached.
*
* <p>This method is NOT threadsafe.
*/
public T getResult() throws IOException {
if (reduced == false) {
result = manager.reduce(collectors);
reduced = true;
}
return result;
}
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.VectorEncoding;
import org.apache.lucene.search.comparators.DoubleComparator;
import org.apache.lucene.util.NumericUtils;
/**
* Base class for producing {@link DoubleValues}
@ -115,6 +116,70 @@ public abstract class DoubleValuesSource implements SegmentCacheable {
return new LongDoubleValuesSource(this);
}
/** Convert to {@link LongValuesSource} by calling {@link NumericUtils#doubleToSortableLong} */
public final LongValuesSource toSortableLongDoubleValuesSource() {
return new SortableLongDoubleValuesSource(this);
}
private static class SortableLongDoubleValuesSource extends LongValuesSource {
private final DoubleValuesSource inner;
private SortableLongDoubleValuesSource(DoubleValuesSource inner) {
this.inner = Objects.requireNonNull(inner);
}
@Override
public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
DoubleValues in = inner.getValues(ctx, scores);
return new LongValues() {
@Override
public long longValue() throws IOException {
return NumericUtils.doubleToSortableLong(in.doubleValue());
}
@Override
public boolean advanceExact(int doc) throws IOException {
return in.advanceExact(doc);
}
};
}
@Override
public boolean needsScores() {
return inner.needsScores();
}
@Override
public int hashCode() {
return inner.hashCode();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
SortableLongDoubleValuesSource that = (SortableLongDoubleValuesSource) o;
return Objects.equals(inner, that.inner);
}
@Override
public String toString() {
return "sortableLong(" + inner.toString() + ")";
}
@Override
public LongValuesSource rewrite(IndexSearcher searcher) throws IOException {
return inner.rewrite(searcher).toLongValuesSource();
}
@Override
public boolean isCacheable(LeafReaderContext ctx) {
return false;
}
}
private static class LongDoubleValuesSource extends LongValuesSource {
private final DoubleValuesSource inner;

View File

@ -23,6 +23,7 @@ import java.util.Objects;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.index.FloatVectorValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.knn.KnnCollectorManager;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.VectorUtil;
@ -108,10 +109,13 @@ public class FloatVectorSimilarityQuery extends AbstractVectorSimilarityQuery {
@Override
@SuppressWarnings("resource")
protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitLimit)
protected TopDocs approximateSearch(
LeafReaderContext context,
Bits acceptDocs,
int visitLimit,
KnnCollectorManager knnCollectorManager)
throws IOException {
KnnCollector collector =
new VectorSimilarityCollector(traversalSimilarity, resultSimilarity, visitLimit);
KnnCollector collector = knnCollectorManager.newCollector(visitLimit, context);
context.reader().searchNearestVectors(field, target, collector, acceptDocs);
return collector.topDocs();
}

View File

@ -24,6 +24,10 @@ abstract class HitsThresholdChecker {
/** Implementation of HitsThresholdChecker which allows global hit counting */
private static class GlobalHitsThresholdChecker extends HitsThresholdChecker {
private final LongAdder globalHitCount = new LongAdder();
// Cache whether the threshold has been reached already. It is not volatile or synchronized on
// purpose to contain the overhead of reading the value similarly to what String#hashCode()
// does. This does not affect correctness.
private boolean thresholdReached = false;
GlobalHitsThresholdChecker(int totalHitsThreshold) {
super(totalHitsThreshold);
@ -32,12 +36,17 @@ abstract class HitsThresholdChecker {
@Override
void incrementHitCount() {
globalHitCount.increment();
if (thresholdReached == false) {
globalHitCount.increment();
}
}
@Override
boolean isThresholdReached() {
return globalHitCount.longValue() > getHitsThreshold();
if (thresholdReached) {
return true;
}
return thresholdReached = globalHitCount.longValue() > getHitsThreshold();
}
@Override

View File

@ -671,10 +671,58 @@ public class IndexSearcher {
}
}
/**
* Lower-level search API. Search all leaves using the given {@link CollectorOwner}, without
* calling {@link CollectorOwner#getResult()} so that clients can reduce and read results
* themselves.
*
* <p>Note that this method doesn't return anything - users can access results by calling {@link
* CollectorOwner#getResult()}
*
* @lucene.experimental
*/
public <C extends Collector> void search(Query query, CollectorOwner<C, ?> collectorOwner)
throws IOException {
final C firstCollector = collectorOwner.newCollector();
query = rewrite(query, firstCollector.scoreMode().needsScores());
final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
search(weight, collectorOwner, firstCollector);
}
private <C extends Collector> void search(
Weight weight, CollectorOwner<C, ?> collectorOwner, C firstCollector) throws IOException {
final LeafSlice[] leafSlices = getSlices();
if (leafSlices.length == 0) {
// there are no segments, nothing to offload to the executor
assert leafContexts.isEmpty();
} else {
final ScoreMode scoreMode = firstCollector.scoreMode();
for (int i = 1; i < leafSlices.length; ++i) {
final C collector = collectorOwner.newCollector();
if (scoreMode != collector.scoreMode()) {
throw new IllegalStateException(
"CollectorManager does not always produce collectors with the same score mode");
}
}
final List<Callable<C>> listTasks = new ArrayList<>(leafSlices.length);
for (int i = 0; i < leafSlices.length; ++i) {
final LeafReaderContext[] leaves = leafSlices[i].leaves;
final C collector = collectorOwner.getCollector(i);
listTasks.add(
() -> {
search(Arrays.asList(leaves), weight, collector);
return collector;
});
}
taskExecutor.invokeAll(listTasks);
}
}
/**
* Lower-level search API.
*
* <p>{@link LeafCollector#collect(int)} is called for every document. <br>
* <p>{@link #searchLeaf(LeafReaderContext, Weight, Collector)} is called for every leaf
* partition. <br>
*
* <p>NOTE: this method executes the searches on all given leaves exclusively. To search across
* all the searchers leaves use {@link #leafContexts}.
@ -694,40 +742,56 @@ public class IndexSearcher {
// threaded...? the Collector could be sync'd?
// always use single thread:
for (LeafReaderContext ctx : leaves) { // search each subreader
final LeafCollector leafCollector;
searchLeaf(ctx, weight, collector);
}
}
/**
* Lower-level search API
*
* <p>{@link LeafCollector#collect(int)} is called for every document. <br>
*
* @param ctx the leaf to execute the search against
* @param weight to match document
* @param collector to receive hits
* @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
* clauses.
*/
protected void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector)
throws IOException {
final LeafCollector leafCollector;
try {
leafCollector = collector.getLeafCollector(ctx);
} catch (
@SuppressWarnings("unused")
CollectionTerminatedException e) {
// there is no doc of interest in this reader context
// continue with the following leaf
return;
}
ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx);
if (scorerSupplier != null) {
scorerSupplier.setTopLevelScoringClause();
BulkScorer scorer = scorerSupplier.bulkScorer();
if (queryTimeout != null) {
scorer = new TimeLimitingBulkScorer(scorer, queryTimeout);
}
try {
leafCollector = collector.getLeafCollector(ctx);
scorer.score(leafCollector, ctx.reader().getLiveDocs());
} catch (
@SuppressWarnings("unused")
CollectionTerminatedException e) {
// there is no doc of interest in this reader context
// collection was terminated prematurely
// continue with the following leaf
continue;
} catch (
@SuppressWarnings("unused")
TimeLimitingBulkScorer.TimeExceededException e) {
partialResult = true;
}
ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx);
if (scorerSupplier != null) {
scorerSupplier.setTopLevelScoringClause();
BulkScorer scorer = scorerSupplier.bulkScorer();
if (queryTimeout != null) {
scorer = new TimeLimitingBulkScorer(scorer, queryTimeout);
}
try {
scorer.score(leafCollector, ctx.reader().getLiveDocs());
} catch (
@SuppressWarnings("unused")
CollectionTerminatedException e) {
// collection was terminated prematurely
// continue with the following leaf
} catch (
@SuppressWarnings("unused")
TimeLimitingBulkScorer.TimeExceededException e) {
partialResult = true;
}
}
// Note: this is called if collection ran successfully, including the above special cases of
// CollectionTerminatedException and TimeExceededException, but no other exception.
leafCollector.finish();
}
// Note: this is called if collection ran successfully, including the above special cases of
// CollectionTerminatedException and TimeExceededException, but no other exception.
leafCollector.finish();
}
/**

View File

@ -20,7 +20,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.document.KnnByteVectorField;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.LeafReader;
@ -52,7 +52,7 @@ public class KnnByteVectorQuery extends AbstractKnnVectorQuery {
* Find the <code>k</code> nearest documents to the target vector according to the vectors in the
* given field. <code>target</code> vector.
*
* @param field a field that has been indexed as a {@link KnnFloatVectorField}.
* @param field a field that has been indexed as a {@link KnnByteVectorField}.
* @param target the target of the search
* @param k the number of documents to find
* @throws IllegalArgumentException if <code>k</code> is less than 1
@ -65,7 +65,7 @@ public class KnnByteVectorQuery extends AbstractKnnVectorQuery {
* Find the <code>k</code> nearest documents to the target vector according to the vectors in the
* given field. <code>target</code> vector.
*
* @param field a field that has been indexed as a {@link KnnFloatVectorField}.
* @param field a field that has been indexed as a {@link KnnByteVectorField}.
* @param target the target of the search
* @param k the number of documents to find
* @param filter a filter applied before the vector search

View File

@ -25,7 +25,7 @@ import org.apache.lucene.util.MathUtil;
final class MaxScoreBulkScorer extends BulkScorer {
static final int INNER_WINDOW_SIZE = 1 << 11;
static final int INNER_WINDOW_SIZE = 1 << 12;
private final int maxDoc;
// All scorers, sorted by increasing max score.
@ -271,11 +271,6 @@ final class MaxScoreBulkScorer extends BulkScorer {
windowMax = (int) Math.min(windowMax, upTo + 1L); // upTo is inclusive
}
// Score at least an entire inner window of docs
windowMax =
Math.max(
windowMax, (int) Math.min(Integer.MAX_VALUE, (long) windowMin + INNER_WINDOW_SIZE));
return windowMax;
}

View File

@ -21,8 +21,8 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
@ -399,10 +399,10 @@ public class PhraseQuery extends Query {
/**
* A guess of the average number of simple operations for the initial seek and buffer refill per
* document for the positions of a term. See also {@link
* Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()}.
* Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()}.
*
* <p>Aside: Instead of being constant this could depend among others on {@link
* Lucene99PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link
* Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link
* TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs),
* {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block
* size of the device storing the index.
@ -411,7 +411,7 @@ public class PhraseQuery extends Query {
/**
* Number of simple operations in {@link
* Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill
* Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill
* is done.
*/
private static final int TERM_OPS_PER_POS = 7;

Some files were not shown because too many files have changed in this diff Show More