LUCENE-9705: Create Lucene90PostingsFormat (#2310)

For now this is just a copy of Lucene90PostingsFormat. The existing
Lucene84PostingsFormat was moved to backwards-codecs, along with its utility
classes.
This commit is contained in:
Julie Tibshirani 2021-02-22 10:45:13 -08:00 committed by GitHub
parent f783848e71
commit f43fe7642e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
90 changed files with 11467 additions and 285 deletions

View File

@ -0,0 +1,72 @@
/*
* 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.lucene40.blocktree;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.util.compress.LowercaseAsciiCompression;
/** Compression algorithm used for suffixes of a block of terms. */
enum CompressionAlgorithm {
NO_COMPRESSION(0x00) {
@Override
void read(DataInput in, byte[] out, int len) throws IOException {
in.readBytes(out, 0, len);
}
},
LOWERCASE_ASCII(0x01) {
@Override
void read(DataInput in, byte[] out, int len) throws IOException {
LowercaseAsciiCompression.decompress(in, out, len);
}
},
LZ4(0x02) {
@Override
void read(DataInput in, byte[] out, int len) throws IOException {
org.apache.lucene.util.compress.LZ4.decompress(in, len, out, 0);
}
};
private static final CompressionAlgorithm[] BY_CODE = new CompressionAlgorithm[3];
static {
for (CompressionAlgorithm alg : CompressionAlgorithm.values()) {
BY_CODE[alg.code] = alg;
}
}
/** Look up a {@link CompressionAlgorithm} by its {@link CompressionAlgorithm#code}. */
static final CompressionAlgorithm byCode(int code) {
if (code < 0 || code >= BY_CODE.length) {
throw new IllegalArgumentException("Illegal code for a compression algorithm: " + code);
}
return BY_CODE[code];
}
public final int code;
private CompressionAlgorithm(int code) {
this.code = code;
}
abstract void read(DataInput in, byte[] out, int len) throws IOException;
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.backward_codecs.lucene40.blocktree;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
@ -56,13 +56,13 @@ public final class FieldReader extends Terms implements Accountable {
final BytesRef rootCode; final BytesRef rootCode;
final BytesRef minTerm; final BytesRef minTerm;
final BytesRef maxTerm; final BytesRef maxTerm;
final BlockTreeTermsReader parent; final Lucene40BlockTreeTermsReader parent;
final FST<BytesRef> index; final FST<BytesRef> index;
// private boolean DEBUG; // private boolean DEBUG;
FieldReader( FieldReader(
BlockTreeTermsReader parent, Lucene40BlockTreeTermsReader parent,
FieldInfo fieldInfo, FieldInfo fieldInfo,
long numTerms, long numTerms,
BytesRef rootCode, BytesRef rootCode,
@ -92,7 +92,7 @@ public final class FieldReader extends Terms implements Accountable {
// } // }
rootBlockFP = rootBlockFP =
(new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong()
>>> BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS; >>> Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
// Initialize FST always off-heap. // Initialize FST always off-heap.
final IndexInput clone = indexIn.clone(); final IndexInput clone = indexIn.clone();
clone.seek(indexStartFP); clone.seek(indexStartFP);

View File

@ -0,0 +1,590 @@
/*
* 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.lucene40.blocktree;
import java.io.IOException;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
/**
* This is used to implement efficient {@link Terms#intersect} for block-tree. Note that it cannot
* seek, except for the initial term on init. It just "nexts" through the intersection of the
* automaton and the terms. It does not use the terms index at all: on init, it loads the root
* block, and scans its way to the initial term. Likewise, in next it scans until it finds a term
* that matches the current automaton transition.
*/
final class IntersectTermsEnum extends BaseTermsEnum {
// static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
final IndexInput in;
static final Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
IntersectTermsEnumFrame[] stack;
@SuppressWarnings({"rawtypes", "unchecked"})
private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
final RunAutomaton runAutomaton;
final Automaton automaton;
final BytesRef commonSuffix;
private IntersectTermsEnumFrame currentFrame;
private Transition currentTransition;
private final BytesRef term = new BytesRef();
private final FST.BytesReader fstReader;
final FieldReader fr;
private BytesRef savedStartTerm;
// TODO: in some cases we can filter by length? eg
// regexp foo*bar must be at least length 6 bytes
public IntersectTermsEnum(
FieldReader fr,
Automaton automaton,
RunAutomaton runAutomaton,
BytesRef commonSuffix,
BytesRef startTerm)
throws IOException {
this.fr = fr;
assert automaton != null;
assert runAutomaton != null;
this.runAutomaton = runAutomaton;
this.automaton = automaton;
this.commonSuffix = commonSuffix;
in = fr.parent.termsIn.clone();
stack = new IntersectTermsEnumFrame[5];
for (int idx = 0; idx < stack.length; idx++) {
stack[idx] = new IntersectTermsEnumFrame(this, idx);
}
for (int arcIdx = 0; arcIdx < arcs.length; arcIdx++) {
arcs[arcIdx] = new FST.Arc<>();
}
fstReader = fr.index.getBytesReader();
// TODO: if the automaton is "smallish" we really
// should use the terms index to seek at least to
// the initial term and likely to subsequent terms
// (or, maybe just fallback to ATE for such cases).
// Else the seek cost of loading the frames will be
// too costly.
final FST.Arc<BytesRef> arc = fr.index.getFirstArc(arcs[0]);
// Empty string prefix must have an output in the index!
assert arc.isFinal();
// Special pushFrame since it's the first one:
final IntersectTermsEnumFrame f = stack[0];
f.fp = f.fpOrig = fr.rootBlockFP;
f.prefix = 0;
f.setState(0);
f.arc = arc;
f.outputPrefix = arc.output();
f.load(fr.rootCode);
// for assert:
assert setSavedStartTerm(startTerm);
currentFrame = f;
if (startTerm != null) {
seekToStartTerm(startTerm);
}
currentTransition = currentFrame.transition;
}
// only for assert:
private boolean setSavedStartTerm(BytesRef startTerm) {
savedStartTerm = startTerm == null ? null : BytesRef.deepCopyOf(startTerm);
return true;
}
@Override
public TermState termState() throws IOException {
currentFrame.decodeMetaData();
return currentFrame.termState.clone();
}
private IntersectTermsEnumFrame getFrame(int ord) throws IOException {
if (ord >= stack.length) {
final IntersectTermsEnumFrame[] next =
new IntersectTermsEnumFrame
[ArrayUtil.oversize(1 + ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(stack, 0, next, 0, stack.length);
for (int stackOrd = stack.length; stackOrd < next.length; stackOrd++) {
next[stackOrd] = new IntersectTermsEnumFrame(this, stackOrd);
}
stack = next;
}
assert stack[ord].ord == ord;
return stack[ord];
}
private FST.Arc<BytesRef> getArc(int ord) {
if (ord >= arcs.length) {
@SuppressWarnings({"rawtypes", "unchecked"})
final FST.Arc<BytesRef>[] next =
new FST.Arc[ArrayUtil.oversize(1 + ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(arcs, 0, next, 0, arcs.length);
for (int arcOrd = arcs.length; arcOrd < next.length; arcOrd++) {
next[arcOrd] = new FST.Arc<>();
}
arcs = next;
}
return arcs[ord];
}
private IntersectTermsEnumFrame pushFrame(int state) throws IOException {
assert currentFrame != null;
final IntersectTermsEnumFrame f = getFrame(currentFrame == null ? 0 : 1 + currentFrame.ord);
f.fp = f.fpOrig = currentFrame.lastSubFP;
f.prefix = currentFrame.prefix + currentFrame.suffix;
f.setState(state);
// Walk the arc through the index -- we only
// "bother" with this so we can get the floor data
// from the index and skip floor blocks when
// possible:
FST.Arc<BytesRef> arc = currentFrame.arc;
int idx = currentFrame.prefix;
assert currentFrame.suffix > 0;
BytesRef output = currentFrame.outputPrefix;
while (idx < f.prefix) {
final int target = term.bytes[idx] & 0xff;
// TODO: we could be more efficient for the next()
// case by using current arc as starting point,
// passed to findTargetArc
arc = fr.index.findTargetArc(target, arc, getArc(1 + idx), fstReader);
assert arc != null;
output = fstOutputs.add(output, arc.output());
idx++;
}
f.arc = arc;
f.outputPrefix = output;
assert arc.isFinal();
f.load(fstOutputs.add(output, arc.nextFinalOutput()));
return f;
}
@Override
public BytesRef term() {
return term;
}
@Override
public int docFreq() throws IOException {
currentFrame.decodeMetaData();
return currentFrame.termState.docFreq;
}
@Override
public long totalTermFreq() throws IOException {
currentFrame.decodeMetaData();
return currentFrame.termState.totalTermFreq;
}
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
currentFrame.decodeMetaData();
return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, reuse, flags);
}
@Override
public ImpactsEnum impacts(int flags) throws IOException {
currentFrame.decodeMetaData();
return fr.parent.postingsReader.impacts(fr.fieldInfo, currentFrame.termState, flags);
}
private int getState() {
int state = currentFrame.state;
for (int idx = 0; idx < currentFrame.suffix; idx++) {
state =
runAutomaton.step(
state, currentFrame.suffixBytes[currentFrame.startBytePos + idx] & 0xff);
assert state != -1;
}
return state;
}
// NOTE: specialized to only doing the first-time
// seek, but we could generalize it to allow
// arbitrary seekExact/Ceil. Note that this is a
// seekFloor!
private void seekToStartTerm(BytesRef target) throws IOException {
assert currentFrame.ord == 0;
if (term.length < target.length) {
term.bytes = ArrayUtil.grow(term.bytes, target.length);
}
FST.Arc<BytesRef> arc = arcs[0];
assert arc == currentFrame.arc;
for (int idx = 0; idx <= target.length; idx++) {
while (true) {
final int savNextEnt = currentFrame.nextEnt;
final int savePos = currentFrame.suffixesReader.getPosition();
final int saveLengthPos = currentFrame.suffixLengthsReader.getPosition();
final int saveStartBytePos = currentFrame.startBytePos;
final int saveSuffix = currentFrame.suffix;
final long saveLastSubFP = currentFrame.lastSubFP;
final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
final boolean isSubBlock = currentFrame.next();
term.length = currentFrame.prefix + currentFrame.suffix;
if (term.bytes.length < term.length) {
term.bytes = ArrayUtil.grow(term.bytes, term.length);
}
System.arraycopy(
currentFrame.suffixBytes,
currentFrame.startBytePos,
term.bytes,
currentFrame.prefix,
currentFrame.suffix);
if (isSubBlock && StringHelper.startsWith(target, term)) {
// Recurse
currentFrame = pushFrame(getState());
break;
} else {
final int cmp = term.compareTo(target);
if (cmp < 0) {
if (currentFrame.nextEnt == currentFrame.entCount) {
if (!currentFrame.isLastInFloor) {
// Advance to next floor block
currentFrame.loadNextFloorBlock();
continue;
} else {
return;
}
}
continue;
} else if (cmp == 0) {
return;
} else {
// Fallback to prior entry: the semantics of
// this method is that the first call to
// next() will return the term after the
// requested term
currentFrame.nextEnt = savNextEnt;
currentFrame.lastSubFP = saveLastSubFP;
currentFrame.startBytePos = saveStartBytePos;
currentFrame.suffix = saveSuffix;
currentFrame.suffixesReader.setPosition(savePos);
currentFrame.suffixLengthsReader.setPosition(saveLengthPos);
currentFrame.termState.termBlockOrd = saveTermBlockOrd;
System.arraycopy(
currentFrame.suffixBytes,
currentFrame.startBytePos,
term.bytes,
currentFrame.prefix,
currentFrame.suffix);
term.length = currentFrame.prefix + currentFrame.suffix;
// If the last entry was a block we don't
// need to bother recursing and pushing to
// the last term under it because the first
// next() will simply skip the frame anyway
return;
}
}
}
}
assert false;
}
private boolean popPushNext() throws IOException {
// Pop finished frames
while (currentFrame.nextEnt == currentFrame.entCount) {
if (!currentFrame.isLastInFloor) {
// Advance to next floor block
currentFrame.loadNextFloorBlock();
break;
} else {
if (currentFrame.ord == 0) {
throw NoMoreTermsException.INSTANCE;
}
final long lastFP = currentFrame.fpOrig;
currentFrame = stack[currentFrame.ord - 1];
currentTransition = currentFrame.transition;
assert currentFrame.lastSubFP == lastFP;
}
}
return currentFrame.next();
}
// Only used internally when there are no more terms in next():
private static final class NoMoreTermsException extends RuntimeException {
// Only used internally when there are no more terms in next():
public static final NoMoreTermsException INSTANCE = new NoMoreTermsException();
private NoMoreTermsException() {}
@Override
public Throwable fillInStackTrace() {
// Do nothing:
return this;
}
}
@Override
public BytesRef next() throws IOException {
try {
return _next();
} catch (NoMoreTermsException eoi) {
// Provoke NPE if we are (illegally!) called again:
currentFrame = null;
return null;
}
}
private BytesRef _next() throws IOException {
boolean isSubBlock = popPushNext();
nextTerm:
while (true) {
assert currentFrame.transition == currentTransition;
int state;
int lastState;
// NOTE: suffix == 0 can only happen on the first term in a block, when
// there is a term exactly matching a prefix in the index. If we
// could somehow re-org the code so we only checked this case immediately
// after pushing a frame...
if (currentFrame.suffix != 0) {
final byte[] suffixBytes = currentFrame.suffixBytes;
// This is the first byte of the suffix of the term we are now on:
final int label = suffixBytes[currentFrame.startBytePos] & 0xff;
if (label < currentTransition.min) {
// Common case: we are scanning terms in this block to "catch up" to
// current transition in the automaton:
int minTrans = currentTransition.min;
while (currentFrame.nextEnt < currentFrame.entCount) {
isSubBlock = currentFrame.next();
if ((suffixBytes[currentFrame.startBytePos] & 0xff) >= minTrans) {
continue nextTerm;
}
}
// End of frame:
isSubBlock = popPushNext();
continue nextTerm;
}
// Advance where we are in the automaton to match this label:
while (label > currentTransition.max) {
if (currentFrame.transitionIndex >= currentFrame.transitionCount - 1) {
// Pop this frame: no further matches are possible because
// we've moved beyond what the max transition will allow
if (currentFrame.ord == 0) {
// Provoke NPE if we are (illegally!) called again:
currentFrame = null;
return null;
}
currentFrame = stack[currentFrame.ord - 1];
currentTransition = currentFrame.transition;
isSubBlock = popPushNext();
continue nextTerm;
}
currentFrame.transitionIndex++;
automaton.getNextTransition(currentTransition);
if (label < currentTransition.min) {
int minTrans = currentTransition.min;
while (currentFrame.nextEnt < currentFrame.entCount) {
isSubBlock = currentFrame.next();
if ((suffixBytes[currentFrame.startBytePos] & 0xff) >= minTrans) {
continue nextTerm;
}
}
// End of frame:
isSubBlock = popPushNext();
continue nextTerm;
}
}
if (commonSuffix != null && !isSubBlock) {
final int termLen = currentFrame.prefix + currentFrame.suffix;
if (termLen < commonSuffix.length) {
// No match
isSubBlock = popPushNext();
continue nextTerm;
}
final byte[] commonSuffixBytes = commonSuffix.bytes;
final int lenInPrefix = commonSuffix.length - currentFrame.suffix;
assert commonSuffix.offset == 0;
int suffixBytesPos;
int commonSuffixBytesPos = 0;
if (lenInPrefix > 0) {
// A prefix of the common suffix overlaps with
// the suffix of the block prefix so we first
// test whether the prefix part matches:
final byte[] termBytes = term.bytes;
int termBytesPos = currentFrame.prefix - lenInPrefix;
assert termBytesPos >= 0;
final int termBytesPosEnd = currentFrame.prefix;
while (termBytesPos < termBytesPosEnd) {
if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
isSubBlock = popPushNext();
continue nextTerm;
}
}
suffixBytesPos = currentFrame.startBytePos;
} else {
suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - commonSuffix.length;
}
// Test overlapping suffix part:
final int commonSuffixBytesPosEnd = commonSuffix.length;
while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
isSubBlock = popPushNext();
continue nextTerm;
}
}
}
// TODO: maybe we should do the same linear test
// that AutomatonTermsEnum does, so that if we
// reach a part of the automaton where .* is
// "temporarily" accepted, we just blindly .next()
// until the limit
// See if the term suffix matches the automaton:
// We know from above that the first byte in our suffix (label) matches
// the current transition, so we step from the 2nd byte
// in the suffix:
lastState = currentFrame.state;
state = currentTransition.dest;
int end = currentFrame.startBytePos + currentFrame.suffix;
for (int idx = currentFrame.startBytePos + 1; idx < end; idx++) {
lastState = state;
state = runAutomaton.step(state, suffixBytes[idx] & 0xff);
if (state == -1) {
// No match
isSubBlock = popPushNext();
continue nextTerm;
}
}
} else {
state = currentFrame.state;
lastState = currentFrame.lastState;
}
if (isSubBlock) {
// Match! Recurse:
copyTerm();
currentFrame = pushFrame(state);
currentTransition = currentFrame.transition;
currentFrame.lastState = lastState;
} else if (runAutomaton.isAccept(state)) {
copyTerm();
assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0
: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
return term;
} else {
// This term is a prefix of a term accepted by the automaton, but is not itself accepted
}
isSubBlock = popPushNext();
}
}
// for debugging
@SuppressWarnings("unused")
static String brToString(BytesRef b) {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
// If BytesRef isn't actually UTF8, or it's eg a
// prefix of UTF8 that ends mid-unicode-char, we
// fallback to hex:
return b.toString();
}
}
private void copyTerm() {
final int len = currentFrame.prefix + currentFrame.suffix;
if (term.bytes.length < len) {
term.bytes = ArrayUtil.grow(term.bytes, len);
}
System.arraycopy(
currentFrame.suffixBytes,
currentFrame.startBytePos,
term.bytes,
currentFrame.prefix,
currentFrame.suffix);
term.length = len;
}
@Override
public boolean seekExact(BytesRef text) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
@Override
public SeekStatus seekCeil(BytesRef text) {
throw new UnsupportedOperationException();
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.backward_codecs.lucene40.blocktree;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
@ -105,7 +105,7 @@ final class IntersectTermsEnumFrame {
this.termState = ite.fr.parent.postingsReader.newTermState(); this.termState = ite.fr.parent.postingsReader.newTermState();
this.termState.totalTermFreq = -1; this.termState.totalTermFreq = -1;
this.version = ite.fr.parent.version; this.version = ite.fr.parent.version;
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
suffixLengthBytes = new byte[32]; suffixLengthBytes = new byte[32];
suffixLengthsReader = new ByteArrayDataInput(); suffixLengthsReader = new ByteArrayDataInput();
} else { } else {
@ -154,7 +154,7 @@ final class IntersectTermsEnumFrame {
// Skip first long -- has redundant fp, hasTerms // Skip first long -- has redundant fp, hasTerms
// flag, isFloor flag // flag, isFloor flag
final long code = floorDataReader.readVLong(); final long code = floorDataReader.readVLong();
if ((code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) { if ((code & Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) {
// Floor frame // Floor frame
numFollowFloorBlocks = floorDataReader.readVInt(); numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff; nextFloorLabel = floorDataReader.readByte() & 0xff;
@ -184,7 +184,7 @@ final class IntersectTermsEnumFrame {
isLastInFloor = (code & 1) != 0; isLastInFloor = (code & 1) != 0;
// term suffixes: // term suffixes:
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
final long codeL = ite.in.readVLong(); final long codeL = ite.in.readVLong();
isLeafBlock = (codeL & 0x04) != 0; isLeafBlock = (codeL & 0x04) != 0;
final int numSuffixBytes = (int) (codeL >>> 3); final int numSuffixBytes = (int) (codeL >>> 3);
@ -315,14 +315,15 @@ final class IntersectTermsEnumFrame {
// just skipN here: // just skipN here:
// stats // stats
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
if (statsSingletonRunLength > 0) { if (statsSingletonRunLength > 0) {
termState.docFreq = 1; termState.docFreq = 1;
termState.totalTermFreq = 1; termState.totalTermFreq = 1;
statsSingletonRunLength--; statsSingletonRunLength--;
} else { } else {
int token = statsReader.readVInt(); int token = statsReader.readVInt();
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES && (token & 1) == 1) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES
&& (token & 1) == 1) {
termState.docFreq = 1; termState.docFreq = 1;
termState.totalTermFreq = 1; termState.totalTermFreq = 1;
statsSingletonRunLength = token >>> 1; statsSingletonRunLength = token >>> 1;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.backward_codecs.lucene40.blocktree;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -60,11 +60,11 @@ import org.apache.lucene.util.fst.Outputs;
* <p>Use {@link org.apache.lucene.index.CheckIndex} with the <code>-verbose</code> option to see * <p>Use {@link org.apache.lucene.index.CheckIndex} with the <code>-verbose</code> option to see
* summary statistics on the blocks in the dictionary. * summary statistics on the blocks in the dictionary.
* *
* <p>See {@link BlockTreeTermsWriter}. * <p>See {@code BlockTreeTermsWriter}.
* *
* @lucene.experimental * @lucene.experimental
*/ */
public final class BlockTreeTermsReader extends FieldsProducer { public final class Lucene40BlockTreeTermsReader extends FieldsProducer {
static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton(); static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton();
@ -124,7 +124,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
final int version; final int version;
/** Sole constructor. */ /** Sole constructor. */
public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) public Lucene40BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state)
throws IOException { throws IOException {
boolean success = false; boolean success = false;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.backward_codecs.lucene40.blocktree;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
@ -102,7 +102,7 @@ final class SegmentTermsEnumFrame {
this.state = ste.fr.parent.postingsReader.newTermState(); this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1; this.state.totalTermFreq = -1;
this.version = ste.fr.parent.version; this.version = ste.fr.parent.version;
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
suffixLengthBytes = new byte[32]; suffixLengthBytes = new byte[32];
suffixLengthsReader = new ByteArrayDataInput(); suffixLengthsReader = new ByteArrayDataInput();
} else { } else {
@ -179,7 +179,7 @@ final class SegmentTermsEnumFrame {
final long startSuffixFP = ste.in.getFilePointer(); final long startSuffixFP = ste.in.getFilePointer();
// term suffixes: // term suffixes:
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
final long codeL = ste.in.readVLong(); final long codeL = ste.in.readVLong();
isLeafBlock = (codeL & 0x04) != 0; isLeafBlock = (codeL & 0x04) != 0;
final int numSuffixBytes = (int) (codeL >>> 3); final int numSuffixBytes = (int) (codeL >>> 3);
@ -471,7 +471,7 @@ final class SegmentTermsEnumFrame {
// TODO: if docFreq were bulk decoded we could // TODO: if docFreq were bulk decoded we could
// just skipN here: // just skipN here:
if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) {
if (statsSingletonRunLength > 0) { if (statsSingletonRunLength > 0) {
state.docFreq = 1; state.docFreq = 1;
state.totalTermFreq = 1; state.totalTermFreq = 1;

View File

@ -0,0 +1,285 @@
/*
* 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.lucene40.blocktree;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.util.Locale;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* BlockTree statistics for a single field returned by {@link FieldReader#getStats()}.
*
* @lucene.internal
*/
public class Stats {
/** Byte size of the index. */
public long indexNumBytes;
/** Total number of terms in the field. */
public long totalTermCount;
/** Total number of bytes (sum of term lengths) across all terms in the field. */
public long totalTermBytes;
/** The number of normal (non-floor) blocks in the terms file. */
public int nonFloorBlockCount;
/**
* The number of floor blocks (meta-blocks larger than the allowed {@code maxItemsPerBlock}) in
* the terms file.
*/
public int floorBlockCount;
/** The number of sub-blocks within the floor blocks. */
public int floorSubBlockCount;
/** The number of "internal" blocks (that have both terms and sub-blocks). */
public int mixedBlockCount;
/** The number of "leaf" blocks (blocks that have only terms). */
public int termsOnlyBlockCount;
/** The number of "internal" blocks that do not contain terms (have only sub-blocks). */
public int subBlocksOnlyBlockCount;
/** Total number of blocks. */
public int totalBlockCount;
/** Number of blocks at each prefix depth. */
public int[] blockCountByPrefixLen = new int[10];
private int startBlockCount;
private int endBlockCount;
/** Total number of bytes used to store term suffixes. */
public long totalBlockSuffixBytes;
/**
* Number of times each compression method has been used. 0 = uncompressed 1 = lowercase_ascii 2 =
* LZ4
*/
public final long[] compressionAlgorithms = new long[3];
/** Total number of suffix bytes before compression. */
public long totalUncompressedBlockSuffixBytes;
/**
* Total number of bytes used to store term stats (not including what the {@link
* PostingsReaderBase} stores.
*/
public long totalBlockStatsBytes;
/**
* Total bytes stored by the {@link PostingsReaderBase}, plus the other few vInts stored in the
* frame.
*/
public long totalBlockOtherBytes;
/** Segment name. */
public final String segment;
/** Field name. */
public final String field;
Stats(String segment, String field) {
this.segment = segment;
this.field = field;
}
void startBlock(SegmentTermsEnumFrame frame, boolean isFloor) {
totalBlockCount++;
if (isFloor) {
if (frame.fp == frame.fpOrig) {
floorBlockCount++;
}
floorSubBlockCount++;
} else {
nonFloorBlockCount++;
}
if (blockCountByPrefixLen.length <= frame.prefix) {
blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1 + frame.prefix);
}
blockCountByPrefixLen[frame.prefix]++;
startBlockCount++;
totalBlockSuffixBytes += frame.totalSuffixBytes;
totalUncompressedBlockSuffixBytes += frame.suffixesReader.length();
if (frame.suffixesReader != frame.suffixLengthsReader) {
totalUncompressedBlockSuffixBytes += frame.suffixLengthsReader.length();
}
totalBlockStatsBytes += frame.statsReader.length();
compressionAlgorithms[frame.compressionAlg.code]++;
}
void endBlock(SegmentTermsEnumFrame frame) {
final int termCount = frame.isLeafBlock ? frame.entCount : frame.state.termBlockOrd;
final int subBlockCount = frame.entCount - termCount;
totalTermCount += termCount;
if (termCount != 0 && subBlockCount != 0) {
mixedBlockCount++;
} else if (termCount != 0) {
termsOnlyBlockCount++;
} else if (subBlockCount != 0) {
subBlocksOnlyBlockCount++;
} else {
throw new IllegalStateException();
}
endBlockCount++;
final long otherBytes =
frame.fpEnd - frame.fp - frame.totalSuffixBytes - frame.statsReader.length();
assert otherBytes > 0
: "otherBytes=" + otherBytes + " frame.fp=" + frame.fp + " frame.fpEnd=" + frame.fpEnd;
totalBlockOtherBytes += otherBytes;
}
void term(BytesRef term) {
totalTermBytes += term.length;
}
void finish() {
assert startBlockCount == endBlockCount
: "startBlockCount=" + startBlockCount + " endBlockCount=" + endBlockCount;
assert totalBlockCount == floorSubBlockCount + nonFloorBlockCount
: "floorSubBlockCount="
+ floorSubBlockCount
+ " nonFloorBlockCount="
+ nonFloorBlockCount
+ " totalBlockCount="
+ totalBlockCount;
assert totalBlockCount == mixedBlockCount + termsOnlyBlockCount + subBlocksOnlyBlockCount
: "totalBlockCount="
+ totalBlockCount
+ " mixedBlockCount="
+ mixedBlockCount
+ " subBlocksOnlyBlockCount="
+ subBlocksOnlyBlockCount
+ " termsOnlyBlockCount="
+ termsOnlyBlockCount;
}
@Override
public String toString() {
final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
PrintStream out;
try {
out = new PrintStream(bos, false, IOUtils.UTF_8);
} catch (UnsupportedEncodingException bogus) {
throw new RuntimeException(bogus);
}
out.println(" index FST:");
out.println(" " + indexNumBytes + " bytes");
out.println(" terms:");
out.println(" " + totalTermCount + " terms");
out.println(
" "
+ totalTermBytes
+ " bytes"
+ (totalTermCount != 0
? " ("
+ String.format(Locale.ROOT, "%.1f", ((double) totalTermBytes) / totalTermCount)
+ " bytes/term)"
: ""));
out.println(" blocks:");
out.println(" " + totalBlockCount + " blocks");
out.println(" " + termsOnlyBlockCount + " terms-only blocks");
out.println(" " + subBlocksOnlyBlockCount + " sub-block-only blocks");
out.println(" " + mixedBlockCount + " mixed blocks");
out.println(" " + floorBlockCount + " floor blocks");
out.println(" " + (totalBlockCount - floorSubBlockCount) + " non-floor blocks");
out.println(" " + floorSubBlockCount + " floor sub-blocks");
out.println(
" "
+ totalUncompressedBlockSuffixBytes
+ " term suffix bytes before compression"
+ (totalBlockCount != 0
? " ("
+ String.format(
Locale.ROOT, "%.1f", ((double) totalBlockSuffixBytes) / totalBlockCount)
+ " suffix-bytes/block)"
: ""));
StringBuilder compressionCounts = new StringBuilder();
for (int code = 0; code < compressionAlgorithms.length; ++code) {
if (compressionAlgorithms[code] == 0) {
continue;
}
if (compressionCounts.length() > 0) {
compressionCounts.append(", ");
}
compressionCounts.append(CompressionAlgorithm.byCode(code));
compressionCounts.append(": ");
compressionCounts.append(compressionAlgorithms[code]);
}
out.println(
" "
+ totalBlockSuffixBytes
+ " compressed term suffix bytes"
+ (totalBlockCount != 0
? " ("
+ String.format(
Locale.ROOT,
"%.2f",
((double) totalBlockSuffixBytes) / totalUncompressedBlockSuffixBytes)
+ " compression ratio - compression count by algorithm: "
+ compressionCounts
: "")
+ ")");
out.println(
" "
+ totalBlockStatsBytes
+ " term stats bytes "
+ (totalBlockCount != 0
? " ("
+ String.format(
Locale.ROOT, "%.1f", ((double) totalBlockStatsBytes) / totalBlockCount)
+ " stats-bytes/block)"
: ""));
out.println(
" "
+ totalBlockOtherBytes
+ " other bytes"
+ (totalBlockCount != 0
? " ("
+ String.format(
Locale.ROOT, "%.1f", ((double) totalBlockOtherBytes) / totalBlockCount)
+ " other-bytes/block)"
: ""));
if (totalBlockCount != 0) {
out.println(" by prefix length:");
int total = 0;
for (int prefix = 0; prefix < blockCountByPrefixLen.length; prefix++) {
final int blockCount = blockCountByPrefixLen[prefix];
total += blockCount;
if (blockCount != 0) {
out.println(" " + String.format(Locale.ROOT, "%2d", prefix) + ": " + blockCount);
}
}
assert totalBlockCount == total;
}
try {
return bos.toString(IOUtils.UTF_8);
} catch (UnsupportedEncodingException bogus) {
throw new RuntimeException(bogus);
}
}
}

View File

@ -23,6 +23,6 @@
* structure. It allows you to plug in your own {@link org.apache.lucene.codecs.PostingsWriterBase} * structure. It allows you to plug in your own {@link org.apache.lucene.codecs.PostingsWriterBase}
* to implement the postings. * to implement the postings.
* *
* <p>See {@link org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter} for the file format. * <p>See {@code BlockTreeTermsWriter} for the file format.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.backward_codecs.lucene40.blocktree;

View File

@ -17,6 +17,7 @@
package org.apache.lucene.backward_codecs.lucene50; package org.apache.lucene.backward_codecs.lucene50;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
@ -24,8 +25,6 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.MultiLevelSkipListWriter; import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -93,7 +92,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <dd><b>Term Dictionary</b> * <dd><b>Term Dictionary</b>
* <p>The .tim file contains the list of terms in each field along with per-term statistics * <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 * (such as docfreq) and pointers to the frequencies, positions, payload and skip data in the
* .doc, .pos, and .pay files. See {@link BlockTreeTermsWriter} for more details on the * .doc, .pos, and .pay files. See {@code BlockTreeTermsWriter} for more details on the
* format. * format.
* <p>NOTE: The term dictionary can plug into different postings implementations: the postings * <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 * writer/reader are actually responsible for encoding and decoding the PostingsHeader and
@ -148,7 +147,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <dl> * <dl>
* <dd><b>Term Index</b> * <dd><b>Term Index</b>
* <p>The .tip file contains an index into the term dictionary, so that it can be accessed * <p>The .tip file contains an index into the term dictionary, so that it can be accessed
* randomly. See {@link BlockTreeTermsWriter} for more details on the format. * randomly. See {@code BlockTreeTermsWriter} for more details on the format.
* </dl> * </dl>
* *
* <a id="Frequencies"></a> * <a id="Frequencies"></a>
@ -387,7 +386,7 @@ public class Lucene50PostingsFormat extends PostingsFormat {
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state); PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false; boolean success = false;
try { try {
FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state); FieldsProducer ret = new Lucene40BlockTreeTermsReader(postingsReader, state);
success = true; success = true;
return ret; return ret;
} finally { } finally {

View File

@ -0,0 +1,86 @@
/*
* 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.lucene84;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.PackedInts;
/** Utility class to encode/decode increasing sequences of 128 integers. */
public class ForDeltaUtil {
// IDENTITY_PLUS_ONE[i] == i+1
private static final long[] IDENTITY_PLUS_ONE = new long[ForUtil.BLOCK_SIZE];
static {
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
IDENTITY_PLUS_ONE[i] = i + 1;
}
}
private static void prefixSumOfOnes(long[] arr, long base) {
System.arraycopy(IDENTITY_PLUS_ONE, 0, arr, 0, ForUtil.BLOCK_SIZE);
// This loop gets auto-vectorized
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
arr[i] += base;
}
}
private final ForUtil forUtil;
ForDeltaUtil(ForUtil forUtil) {
this.forUtil = forUtil;
}
/**
* Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code
* longs} are expected to be deltas between consecutive values.
*/
void encodeDeltas(long[] longs, DataOutput out) throws IOException {
if (longs[0] == 1 && PForUtil.allEqual(longs)) { // happens with very dense postings
out.writeByte((byte) 0);
} else {
long or = 0;
for (long l : longs) {
or |= l;
}
assert or != 0;
final int bitsPerValue = PackedInts.bitsRequired(or);
out.writeByte((byte) bitsPerValue);
forUtil.encode(longs, bitsPerValue, out);
}
}
/** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */
void decodeAndPrefixSum(DataInput in, long base, long[] longs) throws IOException {
final int bitsPerValue = Byte.toUnsignedInt(in.readByte());
if (bitsPerValue == 0) {
prefixSumOfOnes(longs, base);
} else {
forUtil.decodeAndPrefixSum(bitsPerValue, in, base, longs);
}
}
/** Skip a sequence of 128 longs. */
void skip(DataInput in) throws IOException {
final int bitsPerValue = Byte.toUnsignedInt(in.readByte());
if (bitsPerValue != 0) {
in.skipBytes(forUtil.numBytes(bitsPerValue));
}
}
}

View File

@ -39,7 +39,6 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -49,7 +48,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
* *
* <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. * <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
* *
* @see org.apache.lucene.codecs.lucene84 package documentation for file format details. * @see org.apache.lucene.backward_codecs.lucene84 package documentation for file format details.
* @lucene.experimental * @lucene.experimental
*/ */
public class Lucene84Codec extends Codec { public class Lucene84Codec extends Codec {
@ -105,7 +104,7 @@ public class Lucene84Codec extends Codec {
} }
@Override @Override
public final PostingsFormat postingsFormat() { public PostingsFormat postingsFormat() {
return postingsFormat; return postingsFormat;
} }

View File

@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.backward_codecs.lucene84;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
@ -24,9 +25,6 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.MultiLevelSkipListWriter; import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -94,7 +92,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <dd><b>Term Dictionary</b> * <dd><b>Term Dictionary</b>
* <p>The .tim file contains the list of terms in each field along with per-term statistics * <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 * (such as docfreq) and pointers to the frequencies, positions, payload and skip data in the
* .doc, .pos, and .pay files. See {@link BlockTreeTermsWriter} for more details on the * .doc, .pos, and .pay files. See {@code BlockTreeTermsWriter} for more details on the
* format. * format.
* <p>NOTE: The term dictionary can plug into different postings implementations: the postings * <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 * writer/reader are actually responsible for encoding and decoding the PostingsHeader and
@ -149,7 +147,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <dl> * <dl>
* <dd><b>Term Index</b> * <dd><b>Term Index</b>
* <p>The .tip file contains an index into the term dictionary, so that it can be accessed * <p>The .tip file contains an index into the term dictionary, so that it can be accessed
* randomly. See {@link BlockTreeTermsWriter} for more details on the format. * randomly. See {@code BlockTreeTermsWriter} for more details on the format.
* </dl> * </dl>
* *
* <a id="Frequencies"></a> * <a id="Frequencies"></a>
@ -331,7 +329,7 @@ import org.apache.lucene.util.packed.PackedInts;
* *
* @lucene.experimental * @lucene.experimental
*/ */
public final class Lucene84PostingsFormat extends PostingsFormat { public class Lucene84PostingsFormat extends PostingsFormat {
/** /**
* Filename extension for document number, frequencies, and skip data. See chapter: <a * Filename extension for document number, frequencies, and skip data. See chapter: <a
@ -368,25 +366,9 @@ public final class Lucene84PostingsFormat extends PostingsFormat {
static final int VERSION_COMPRESSED_TERMS_DICT_IDS = 1; static final int VERSION_COMPRESSED_TERMS_DICT_IDS = 1;
static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS_DICT_IDS; static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS_DICT_IDS;
private final int minTermBlockSize;
private final int maxTermBlockSize;
/** Creates {@code Lucene84PostingsFormat} with default settings. */ /** Creates {@code Lucene84PostingsFormat} with default settings. */
public Lucene84PostingsFormat() { public Lucene84PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/**
* Creates {@code Lucene84PostingsFormat} with custom values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
*
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)
*/
public Lucene84PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene84"); super("Lucene84");
BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize;
} }
@Override @Override
@ -396,18 +378,7 @@ public final class Lucene84PostingsFormat extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); throw new UnsupportedOperationException("Old codecs may only be used for reading");
boolean success = false;
try {
FieldsConsumer ret =
new BlockTreeTermsWriter(state, postingsWriter, minTermBlockSize, maxTermBlockSize);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
} }
@Override @Override
@ -415,7 +386,7 @@ public final class Lucene84PostingsFormat extends PostingsFormat {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); PostingsReaderBase postingsReader = new Lucene84PostingsReader(state);
boolean success = false; boolean success = false;
try { try {
FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state); FieldsProducer ret = new Lucene40BlockTreeTermsReader(postingsReader, state);
success = true; success = true;
return ret; return ret;
} finally { } finally {

View File

@ -14,24 +14,24 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.backward_codecs.lucene84;
import static org.apache.lucene.codecs.lucene84.ForUtil.BLOCK_SIZE; import static org.apache.lucene.backward_codecs.lucene84.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.DOC_CODEC; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.DOC_CODEC;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.MAX_SKIP_LEVELS; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.PAY_CODEC; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.PAY_CODEC;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.POS_CODEC; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.POS_CODEC;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.TERMS_CODEC; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.VERSION_COMPRESSED_TERMS_DICT_IDS; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.VERSION_COMPRESSED_TERMS_DICT_IDS;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.VERSION_CURRENT; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.VERSION_CURRENT;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.VERSION_START; import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.VERSION_START;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Impacts; import org.apache.lucene.index.Impacts;
import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.ImpactsEnum;
@ -135,12 +135,12 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
state.segmentInfo.getId(), state.segmentInfo.getId(),
state.segmentSuffix); state.segmentSuffix);
final int indexBlockSize = termsIn.readVInt(); final int indexBlockSize = termsIn.readVInt();
if (indexBlockSize != BLOCK_SIZE) { if (indexBlockSize != ForUtil.BLOCK_SIZE) {
throw new IllegalStateException( throw new IllegalStateException(
"index-time BLOCK_SIZE (" "index-time BLOCK_SIZE ("
+ indexBlockSize + indexBlockSize
+ ") != read-time BLOCK_SIZE (" + ") != read-time BLOCK_SIZE ("
+ BLOCK_SIZE + ForUtil.BLOCK_SIZE
+ ")"); + ")");
} }
} }
@ -413,7 +413,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS); this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS);
this.isFreqsRead = true; this.isFreqsRead = true;
if (indexHasFreq == false || needsFreq == false) { if (indexHasFreq == false || needsFreq == false) {
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { for (int i = 0; i < BLOCK_SIZE; ++i) {
freqBuffer[i] = 1; freqBuffer[i] = 1;
} }
} }
@ -1723,7 +1723,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
docFreq); docFreq);
if (indexHasFreq == false) { if (indexHasFreq == false) {
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { for (int i = 0; i < BLOCK_SIZE; ++i) {
freqBuffer[i] = 1; freqBuffer[i] = 1;
} }
} }

View File

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

View File

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

View File

@ -0,0 +1,123 @@
/*
* 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.lucene84;
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.packed.PackedInts;
/** Utility class to encode sequences of 128 small positive integers. */
public final class PForUtil {
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) {
this.forUtil = forUtil;
}
/** Encode 128 integers from {@code longs} into {@code out}. */
void encode(long[] longs, DataOutput out) throws IOException {
// At most 3 exceptions
final long[] top4 = new long[4];
Arrays.fill(top4, -1L);
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
if (longs[i] > top4[0]) {
top4[0] = longs[i];
Arrays.sort(
top4); // For only 4 entries we just sort on every iteration instead of maintaining a PQ
}
}
final int maxBitsRequired = PackedInts.bitsRequired(top4[3]);
// 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(top4[0]), maxBitsRequired - 8);
int numExceptions = 0;
final long maxUnpatchedValue = (1L << patchedBitsRequired) - 1;
for (int i = 1; i < 4; ++i) {
if (top4[i] > maxUnpatchedValue) {
numExceptions++;
}
}
final byte[] exceptions = new byte[numExceptions * 2];
if (numExceptions > 0) {
int exceptionCount = 0;
for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
if (longs[i] > maxUnpatchedValue) {
exceptions[exceptionCount * 2] = (byte) i;
exceptions[exceptionCount * 2 + 1] = (byte) (longs[i] >>> patchedBitsRequired);
longs[i] &= maxUnpatchedValue;
exceptionCount++;
}
}
assert exceptionCount == numExceptions : exceptionCount + " " + numExceptions;
}
if (allEqual(longs) && maxBitsRequired <= 8) {
for (int i = 0; i < numExceptions; ++i) {
exceptions[2 * i + 1] =
(byte) (Byte.toUnsignedLong(exceptions[2 * i + 1]) << patchedBitsRequired);
}
out.writeByte((byte) (numExceptions << 5));
out.writeVLong(longs[0]);
} else {
final int token = (numExceptions << 5) | patchedBitsRequired;
out.writeByte((byte) token);
forUtil.encode(longs, patchedBitsRequired, out);
}
out.writeBytes(exceptions, exceptions.length);
}
/** Decode 128 integers into {@code ints}. */
void decode(DataInput in, long[] longs) throws IOException {
final int token = Byte.toUnsignedInt(in.readByte());
final int bitsPerValue = token & 0x1f;
final int numExceptions = token >>> 5;
if (bitsPerValue == 0) {
Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong());
} else {
forUtil.decode(bitsPerValue, in, longs);
}
for (int i = 0; i < numExceptions; ++i) {
longs[Byte.toUnsignedInt(in.readByte())] |=
Byte.toUnsignedLong(in.readByte()) << bitsPerValue;
}
}
/** Skip 128 integers. */
void skip(DataInput in) throws IOException {
final int token = Byte.toUnsignedInt(in.readByte());
final int bitsPerValue = token & 0x1f;
final int numExceptions = token >>> 5;
if (bitsPerValue == 0) {
in.readVLong();
in.skipBytes((numExceptions << 1));
} else {
in.skipBytes(forUtil.numBytes(bitsPerValue) + (numExceptions << 1));
}
}
}

View File

@ -22,6 +22,7 @@ import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
@ -37,7 +38,6 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -106,7 +106,7 @@ public class Lucene86Codec extends Codec {
} }
@Override @Override
public final PostingsFormat postingsFormat() { public PostingsFormat postingsFormat() {
return postingsFormat; return postingsFormat;
} }

View File

@ -21,6 +21,7 @@ import java.util.Objects;
import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat; import org.apache.lucene.backward_codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
@ -37,7 +38,6 @@ import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
@ -118,7 +118,7 @@ public class Lucene87Codec extends Codec {
} }
@Override @Override
public final PostingsFormat postingsFormat() { public PostingsFormat postingsFormat() {
return postingsFormat; return postingsFormat;
} }

View File

@ -14,3 +14,4 @@
# limitations under the License. # limitations under the License.
org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat
org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat

View File

@ -17,14 +17,31 @@
package org.apache.lucene.backward_codecs.Lucene87; package org.apache.lucene.backward_codecs.Lucene87;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat;
import org.apache.lucene.backward_codecs.lucene87.Lucene87Codec; import org.apache.lucene.backward_codecs.lucene87.Lucene87Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene87Codec}. */ /** RW impersonation of {@link Lucene87Codec}. */
public class Lucene87RWCodec extends Lucene87Codec { public class Lucene87RWCodec extends Lucene87Codec {
private final PostingsFormat defaultPF = new Lucene84RWPostingsFormat();
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPF;
}
};
@Override @Override
public final CompoundFormat compoundFormat() { public final CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat(); return new Lucene50RWCompoundFormat();
} }
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
} }

View File

@ -17,9 +17,9 @@
package org.apache.lucene.backward_codecs.lucene50; package org.apache.lucene.backward_codecs.lucene50;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
@ -31,11 +31,11 @@ public class Lucene50RWPostingsFormat extends Lucene50PostingsFormat {
boolean success = false; boolean success = false;
try { try {
FieldsConsumer ret = FieldsConsumer ret =
new BlockTreeTermsWriter( new Lucene40BlockTreeTermsWriter(
state, state,
postingsWriter, postingsWriter,
BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, Lucene40BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); Lucene40BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
success = true; success = true;
return ret; return ret;
} finally { } finally {

View File

@ -21,11 +21,11 @@ import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.backward_codecs.lucene40.blocktree.FieldReader;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Stats;
import org.apache.lucene.backward_codecs.lucene50.Lucene50ScoreSkipReader.MutableImpactList; import org.apache.lucene.backward_codecs.lucene50.Lucene50ScoreSkipReader.MutableImpactList;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.index.BasePostingsFormatTestCase; import org.apache.lucene.index.BasePostingsFormatTestCase;

View File

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

View File

@ -22,12 +22,23 @@ import org.apache.lucene.backward_codecs.lucene60.Lucene60RWPointsFormat;
import org.apache.lucene.backward_codecs.lucene70.Lucene70RWSegmentInfoFormat; import org.apache.lucene.backward_codecs.lucene70.Lucene70RWSegmentInfoFormat;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointsFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene84Codec}. */ /** RW impersonation of {@link Lucene84Codec}. */
public class Lucene84RWCodec extends Lucene84Codec { public class Lucene84RWCodec extends Lucene84Codec {
private final PostingsFormat defaultPF = new Lucene84RWPostingsFormat();
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPF;
}
};
@Override @Override
public PointsFormat pointsFormat() { public PointsFormat pointsFormat() {
return new Lucene60RWPointsFormat(); return new Lucene60RWPointsFormat();
@ -43,6 +54,11 @@ public class Lucene84RWCodec extends Lucene84Codec {
return new Lucene50RWStoredFieldsFormat(); return new Lucene50RWStoredFieldsFormat();
} }
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override @Override
public final CompoundFormat compoundFormat() { public final CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat(); return new Lucene50RWCompoundFormat();

View File

@ -0,0 +1,46 @@
/*
* 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.lucene84;
import java.io.IOException;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
public class Lucene84RWPostingsFormat extends Lucene84PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret =
new Lucene40BlockTreeTermsWriter(
state,
postingsWriter,
Lucene40BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene40BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
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 * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.backward_codecs.lucene84;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;

View File

@ -0,0 +1,93 @@
/*
* 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.lucene84;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.PackedInts;
public class TestForDeltaUtil extends LuceneTestCase {
public void testEncodeDecode() throws IOException {
final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000);
final int[] values = new int[iterations * ForUtil.BLOCK_SIZE];
for (int i = 0; i < iterations; ++i) {
final int bpv = TestUtil.nextInt(random(), 1, 31 - 7);
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
values[i * ForUtil.BLOCK_SIZE + j] =
RandomNumbers.randomIntBetween(random(), 1, (int) PackedInts.maxValue(bpv));
}
}
final Directory d = new ByteBuffersDirectory();
final long endPointer;
{
// encode
IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(new ForUtil());
for (int i = 0; i < iterations; ++i) {
long[] source = new long[ForUtil.BLOCK_SIZE];
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
source[j] = values[i * ForUtil.BLOCK_SIZE + j];
}
forDeltaUtil.encodeDeltas(source, out);
}
endPointer = out.getFilePointer();
out.close();
}
{
// decode
IndexInput in = d.openInput("test.bin", IOContext.READONCE);
final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(new ForUtil());
for (int i = 0; i < iterations; ++i) {
if (random().nextInt(5) == 0) {
forDeltaUtil.skip(in);
continue;
}
long base = 0;
final long[] restored = new long[ForUtil.BLOCK_SIZE];
forDeltaUtil.decodeAndPrefixSum(in, base, restored);
final long[] expected = new long[ForUtil.BLOCK_SIZE];
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
expected[j] = values[i * ForUtil.BLOCK_SIZE + j];
if (j > 0) {
expected[j] += expected[j - 1];
} else {
expected[j] += base;
}
}
assertArrayEquals(Arrays.toString(restored), expected, restored);
}
assertEquals(endPointer, in.getFilePointer());
in.close();
}
d.close();
}
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene84;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.PackedInts;
public class TestForUtil extends LuceneTestCase {
public void testEncodeDecode() throws IOException {
final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000);
final int[] values = new int[iterations * ForUtil.BLOCK_SIZE];
for (int i = 0; i < iterations; ++i) {
final int bpv = TestUtil.nextInt(random(), 1, 31);
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
values[i * ForUtil.BLOCK_SIZE + j] =
RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv));
}
}
final Directory d = new ByteBuffersDirectory();
final long endPointer;
{
// encode
IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
final ForUtil forUtil = new ForUtil();
for (int i = 0; i < iterations; ++i) {
long[] source = new long[ForUtil.BLOCK_SIZE];
long or = 0;
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
source[j] = values[i * ForUtil.BLOCK_SIZE + j];
or |= source[j];
}
final int bpv = PackedInts.bitsRequired(or);
out.writeByte((byte) bpv);
forUtil.encode(source, bpv, out);
}
endPointer = out.getFilePointer();
out.close();
}
{
// decode
IndexInput in = d.openInput("test.bin", IOContext.READONCE);
final ForUtil forUtil = new ForUtil();
for (int i = 0; i < iterations; ++i) {
final int bitsPerValue = in.readByte();
final long currentFilePointer = in.getFilePointer();
final long[] restored = new long[ForUtil.BLOCK_SIZE];
forUtil.decode(bitsPerValue, in, restored);
int[] ints = new int[ForUtil.BLOCK_SIZE];
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
ints[j] = Math.toIntExact(restored[j]);
}
assertArrayEquals(
Arrays.toString(ints),
ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE),
ints);
assertEquals(forUtil.numBytes(bitsPerValue), in.getFilePointer() - currentFilePointer);
}
assertEquals(endPointer, in.getFilePointer());
in.close();
}
d.close();
}
}

View File

@ -14,18 +14,18 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.backward_codecs.lucene84;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.backward_codecs.lucene40.blocktree.FieldReader;
import org.apache.lucene.backward_codecs.lucene40.blocktree.Stats;
import org.apache.lucene.backward_codecs.lucene84.Lucene84ScoreSkipReader.MutableImpactList;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.codecs.lucene84.Lucene84ScoreSkipReader.MutableImpactList;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.index.BasePostingsFormatTestCase; import org.apache.lucene.index.BasePostingsFormatTestCase;
@ -41,7 +41,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
public class TestLucene84PostingsFormat extends BasePostingsFormatTestCase { public class TestLucene84PostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene84PostingsFormat()); private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene84RWPostingsFormat());
@Override @Override
protected Codec getCodec() { protected Codec getCodec() {
@ -73,22 +73,6 @@ public class TestLucene84PostingsFormat extends BasePostingsFormatTestCase {
d.close(); d.close();
} }
private void shouldFail(int minItemsInBlock, int maxItemsInBlock) {
expectThrows(
IllegalArgumentException.class,
() -> {
new Lucene84PostingsFormat(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 { public void testImpactSerialization() throws IOException {
// omit norms and omit freqs // omit norms and omit freqs
doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L)));

View File

@ -0,0 +1,100 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.backward_codecs.lucene84;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.packed.PackedInts;
public class TestPForUtil extends LuceneTestCase {
public void testEncodeDecode() throws IOException {
final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000);
final int[] values = new int[iterations * ForUtil.BLOCK_SIZE];
for (int i = 0; i < iterations; ++i) {
final int bpv = TestUtil.nextInt(random(), 0, 31);
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
values[i * ForUtil.BLOCK_SIZE + j] =
RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv));
if (random().nextInt(100) == 0) {
final int exceptionBpv;
if (random().nextInt(10) == 0) {
exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 9, 16), 31);
} else {
exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 1, 8), 31);
}
values[i * ForUtil.BLOCK_SIZE + j] |= random().nextInt(1 << (exceptionBpv - bpv)) << bpv;
}
}
}
final Directory d = new ByteBuffersDirectory();
final long endPointer;
{
// encode
IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
final PForUtil pforUtil = new PForUtil(new ForUtil());
for (int i = 0; i < iterations; ++i) {
long[] source = new long[ForUtil.BLOCK_SIZE];
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
source[j] = values[i * ForUtil.BLOCK_SIZE + j];
}
pforUtil.encode(source, out);
}
endPointer = out.getFilePointer();
out.close();
}
{
// decode
IndexInput in = d.openInput("test.bin", IOContext.READONCE);
final PForUtil pforUtil = new PForUtil(new ForUtil());
for (int i = 0; i < iterations; ++i) {
if (random().nextInt(5) == 0) {
pforUtil.skip(in);
continue;
}
final long[] restored = new long[ForUtil.BLOCK_SIZE];
pforUtil.decode(in, restored);
int[] ints = new int[ForUtil.BLOCK_SIZE];
for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) {
ints[j] = Math.toIntExact(restored[j]);
}
assertArrayEquals(
Arrays.toString(ints),
ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE),
ints);
}
assertEquals(endPointer, in.getFilePointer());
in.close();
}
d.close();
}
}

View File

@ -19,13 +19,24 @@ package org.apache.lucene.backward_codecs.lucene86;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWCompoundFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84RWPostingsFormat;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/** RW impersonation of {@link Lucene86Codec}. */ /** RW impersonation of {@link Lucene86Codec}. */
public class Lucene86RWCodec extends Lucene86Codec { public class Lucene86RWCodec extends Lucene86Codec {
private final StoredFieldsFormat storedFieldsFormat; private final StoredFieldsFormat storedFieldsFormat;
private final PostingsFormat defaultPF = new Lucene84RWPostingsFormat();
private final PostingsFormat postingsFormat =
new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPF;
}
};
/** No arguments constructor. */ /** No arguments constructor. */
public Lucene86RWCodec() { public Lucene86RWCodec() {
@ -42,6 +53,11 @@ public class Lucene86RWCodec extends Lucene86Codec {
return storedFieldsFormat; return storedFieldsFormat;
} }
@Override
public PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override @Override
public final CompoundFormat compoundFormat() { public final CompoundFormat compoundFormat() {
return new Lucene50RWCompoundFormat(); return new Lucene50RWCompoundFormat();

View File

@ -22,14 +22,14 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene84PostingsWriter}. */ /** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene90PostingsWriter}. */
public class BlockTreeOrdsPostingsFormat extends PostingsFormat { public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
private final int minTermBlockSize; private final int minTermBlockSize;
@ -57,7 +57,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
super("BlockTreeOrds"); super("BlockTreeOrds");
this.minTermBlockSize = minTermBlockSize; this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize; this.maxTermBlockSize = maxTermBlockSize;
BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
} }
@Override @Override
@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state);
boolean success = false; boolean success = false;
try { try {
@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); PostingsReaderBase postingsReader = new Lucene90PostingsReader(state);
boolean success = false; boolean success = false;
try { try {
FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state); FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state);

View File

@ -24,8 +24,8 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; // javadocs
import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output; import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; // javadocs
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields; import org.apache.lucene.index.Fields;
@ -79,8 +79,8 @@ import org.apache.lucene.util.fst.Util;
*/ */
/** /**
* This is just like {@link BlockTreeTermsWriter}, except it also stores a version per term, and * This is just like {@link Lucene90BlockTreeTermsWriter}, except it also stores a version per term,
* adds a method to its TermsEnum implementation to seekExact only if the version is &gt;= the * and adds a method to its TermsEnum implementation to seekExact only if the version is &gt;= the
* specified version. The version is added to the terms index to avoid seeking if no term in the * specified version. The version is added to the terms index to avoid seeking if no term in the
* block has a high enough version. The term blocks file is .tiv and the terms index extension is * block has a high enough version. The term blocks file is .tiv and the terms index extension is
* .tipv. * .tipv.
@ -188,7 +188,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
int minItemsInBlock, int minItemsInBlock,
int maxItemsInBlock) int maxItemsInBlock)
throws IOException { throws IOException {
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); Lucene90BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
maxDoc = state.segmentInfo.maxDoc(); maxDoc = state.segmentInfo.maxDoc();

View File

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

View File

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

View File

@ -17,13 +17,13 @@
package org.apache.lucene.codecs.uniformsplit; package org.apache.lucene.codecs.uniformsplit;
import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.BLOCK_SIZE; import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.BLOCK_SIZE;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState; import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.IntBlockTermState;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.TermState; 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 * {@link TermState} serializer which encodes each file pointer as a delta relative to a base file
* pointer. It differs from {@link Lucene84PostingsWriter#encodeTerm} which encodes each file * pointer. It differs from {@link Lucene90PostingsWriter#encodeTerm} which encodes each file
* pointer as a delta relative to the previous file pointer. * 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, * <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}. * Writes a {@link BlockTermState} to the provided {@link DataOutput}.
* *
* <p>Simpler variant of {@link Lucene84PostingsWriter#encodeTerm(DataOutput, FieldInfo, * <p>Simpler variant of {@link Lucene90PostingsWriter#encodeTerm(DataOutput, FieldInfo,
* BlockTermState, boolean)}. * BlockTermState, boolean)}.
*/ */
public void writeTermState( public void writeTermState(
@ -148,7 +148,7 @@ public class DeltaBaseTermStateSerializer implements Accountable {
/** /**
* Reads a {@link BlockTermState} from the provided {@link DataInput}. * Reads a {@link BlockTermState} from the provided {@link DataInput}.
* *
* <p>Simpler variant of {@link Lucene84PostingsReader#decodeTerm(DataInput, FieldInfo, * <p>Simpler variant of {@link Lucene90PostingsReader#decodeTerm(DataInput, FieldInfo,
* BlockTermState, boolean)}. * BlockTermState, boolean)}.
* *
* @param reuse {@link BlockTermState} to reuse; or null to create a new one. * @param reuse {@link BlockTermState} to reuse; or null to create a new one.

View File

@ -38,7 +38,7 @@ import org.apache.lucene.util.automaton.Transition;
* automaton. * automaton.
* *
* <p>By design of the UniformSplit block keys, it is less efficient than {@code * <p>By design of the UniformSplit block keys, it is less efficient than {@code
* org.apache.lucene.codecs.blocktree.IntersectTermsEnum} for {@link * org.apache.lucene.backward_codecs.lucene40.blocktree.IntersectTermsEnum} for {@link
* org.apache.lucene.search.FuzzyQuery} (-37%). It is slightly slower for {@link * org.apache.lucene.search.FuzzyQuery} (-37%). It is slightly slower for {@link
* org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for {@link * org.apache.lucene.search.WildcardQuery} (-5%) and slightly faster for {@link
* org.apache.lucene.search.PrefixQuery} (+5%). * org.apache.lucene.search.PrefixQuery} (+5%).

View File

@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
@ -110,7 +110,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state);
boolean success = false; boolean success = false;
try { try {
FieldsConsumer termsWriter = FieldsConsumer termsWriter =
@ -127,7 +127,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); PostingsReaderBase postingsReader = new Lucene90PostingsReader(state);
boolean success = false; boolean success = false;
try { try {
FieldsProducer termsReader = FieldsProducer termsReader =

View File

@ -28,7 +28,7 @@
* org.apache.lucene.search.PhraseQuery}) * org.apache.lucene.search.PhraseQuery})
* <li>Quite efficient for {@link org.apache.lucene.search.PrefixQuery} * <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 * <li>Not efficient for spell-check and {@link org.apache.lucene.search.FuzzyQuery}, in this case
* prefer {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat} * prefer {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat}
* </ul> * </ul>
*/ */
package org.apache.lucene.codecs.uniformsplit; package org.apache.lucene.codecs.uniformsplit;

View File

@ -15,13 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.codecs.lucene90;
/** Test utility class to create mock {@link Lucene84PostingsFormat.IntBlockTermState}. */ /** Test utility class to create mock {@link Lucene90PostingsFormat.IntBlockTermState}. */
public class MockTermStateFactory { public class MockTermStateFactory {
/** Creates an empty {@link Lucene84PostingsFormat.IntBlockTermState}. */ /** Creates an empty {@link Lucene90PostingsFormat.IntBlockTermState}. */
public static Lucene84PostingsFormat.IntBlockTermState create() { public static Lucene90PostingsFormat.IntBlockTermState create() {
return new Lucene84PostingsFormat.IntBlockTermState(); return new Lucene90PostingsFormat.IntBlockTermState();
} }
} }

View File

@ -19,7 +19,7 @@ package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import org.apache.lucene.codecs.lucene84.MockTermStateFactory; import org.apache.lucene.codecs.lucene90.MockTermStateFactory;
import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;

View File

@ -27,7 +27,7 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.lucene84.MockTermStateFactory; import org.apache.lucene.codecs.lucene90.MockTermStateFactory;
import org.apache.lucene.codecs.uniformsplit.BlockHeader; import org.apache.lucene.codecs.uniformsplit.BlockHeader;
import org.apache.lucene.codecs.uniformsplit.BlockLine; import org.apache.lucene.codecs.uniformsplit.BlockLine;
import org.apache.lucene.codecs.uniformsplit.FSTDictionary; import org.apache.lucene.codecs.uniformsplit.FSTDictionary;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.codecs;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
@ -28,8 +28,8 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
/** /**
* Class that plugs into term dictionaries, such as {@link BlockTreeTermsWriter}, and handles * Class that plugs into term dictionaries, such as {@link Lucene90BlockTreeTermsWriter}, and
* writing postings. * handles writing postings.
* *
* @see PostingsReaderBase * @see PostingsReaderBase
* @lucene.experimental * @lucene.experimental

View File

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

View File

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

View File

@ -33,7 +33,6 @@ import org.apache.lucene.codecs.VectorFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat; import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat; import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
@ -109,7 +108,7 @@ public class Lucene90Codec extends Codec {
super("Lucene90"); super("Lucene90");
this.storedFieldsFormat = this.storedFieldsFormat =
new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultFormat = new Lucene84PostingsFormat(); this.defaultFormat = new Lucene90PostingsFormat();
this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode); this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode);
} }

View File

@ -0,0 +1,500 @@
/*
* 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.lucene90;
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.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;
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 5.0 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, Lucene90PostingsFormat 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 settings</b>:
* <p>The structure of skip table is quite similar to previous version of Lucene. Skip
* interval is the same as block size, and each skip entry points to the beginning of each
* block. However, for the first block, skip data is omitted.
* <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?, SkipFPDelta?
* <li>Header, --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>PackedBlockSize, SingletonDocID --&gt; {@link DataOutput#writeVInt VInt}
* <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --&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>SkipFPDelta determines the position of this term's SkipData within the .doc file. In
* particular, it is the length of the TermFreq data. SkipDelta is only stored if
* DocFreq is not smaller than SkipMinimum (i.e. 128 in Lucene90PostingsFormat).
* <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}). It also saves skip data to the beginning of each packed or VInt block,
* when the length of document list is larger than packed block size.
* <ul>
* <li>docFile(.doc) --&gt; Header, &lt;TermFreqs, SkipData?&gt;<sup>TermCount</sup>, Footer
* <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
* <li>TermFreqs --&gt; &lt;PackedBlock&gt; <sup>PackedDocBlockNum</sup>, VIntBlock?
* <li>PackedBlock --&gt; PackedDocDeltaBlock, PackedFreqBlock?
* <li>VIntBlock --&gt; &lt;DocDelta[,
* Freq?]&gt;<sup>DocFreq-PackedBlockSize*PackedDocBlockNum</sup>
* <li>SkipData --&gt; &lt;&lt;SkipLevelLength, SkipLevel&gt; <sup>NumSkipLevels-1</sup>,
* SkipLevel&gt;, SkipDatum?
* <li>SkipLevel --&gt; &lt;SkipDatum&gt; <sup>TrimmedDocFreq/(PackedBlockSize^(Level +
* 1))</sup>
* <li>SkipDatum --&gt; DocSkip, DocFPSkip, &lt;PosFPSkip, PosBlockOffset, PayLength?,
* PayFPSkip?&gt;?, SkipChildLevelPointer?
* <li>PackedDocDeltaBlock, PackedFreqBlock --&gt; {@link PackedInts PackedInts}
* <li>DocDelta, Freq, DocSkip, DocFPSkip, PosFPSkip, PosBlockOffset, PayByteUpto, PayFPSkip
* --&gt; {@link DataOutput#writeVInt VInt}
* <li>SkipChildLevelPointer --&gt; {@link DataOutput#writeVLong VLong}
* <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>TrimmedDocFreq = DocFreq % PackedBlockSize == 0 ? DocFreq - 1 : DocFreq. We use this
* trick since the definition of skip entry is a little different from base interface.
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
* skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list.
* However, in Lucene90PostingsFormat, the skip data is saved for
* skipInterval+1<sup>th</sup>, 2*skipInterval+1<sup>th</sup> ... posting
* (skipInterval==PackedBlockSize in this case). When DocFreq is multiple of
* PackedBlockSize, MultiLevelSkipListWriter will expect one more skip data than
* Lucene90SkipWriter.
* <li>SkipDatum is the metadata of one skip entry. For the first block (no matter packed or
* VInt), it is omitted.
* <li>DocSkip records the document number of every PackedBlockSize<sup>th</sup> document
* number in the postings (i.e. last document number in each packed block). On disk it
* is stored as the difference from previous value in the sequence.
* <li>DocFPSkip records the file offsets of each block (excluding )posting at
* PackedBlockSize+1<sup>th</sup>, 2*PackedBlockSize+1<sup>th</sup> ... , in DocFile.
* The file offsets are relative to the start of current term's TermFreqs. On disk it is
* also stored as the difference from previous SkipDatum in the sequence.
* <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
* </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 Lucene90PostingsFormat extends PostingsFormat {
/**
* 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;
/**
* Expert: The maximum number of skip levels. Smaller values result in slightly smaller indexes,
* but slower skipping in big posting lists.
*/
static final int MAX_SKIP_LEVELS = 10;
static final String TERMS_CODEC = "Lucene90PostingsWriterTerms";
static final String DOC_CODEC = "Lucene90PostingsWriterDoc";
static final String POS_CODEC = "Lucene90PostingsWriterPos";
static final String PAY_CODEC = "Lucene90PostingsWriterPay";
// Increment version to change it
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
private final int minTermBlockSize;
private final int maxTermBlockSize;
/** Creates {@code Lucene90PostingsFormat} with default settings. */
public Lucene90PostingsFormat() {
this(
Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/**
* Creates {@code Lucene90PostingsFormat} with custom values for {@code minBlockSize} and {@code
* maxBlockSize} passed to block terms dictionary.
*
* @see
* Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)
*/
public Lucene90PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene90");
Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize);
this.minTermBlockSize = minTermBlockSize;
this.maxTermBlockSize = maxTermBlockSize;
}
@Override
public String toString() {
return getName();
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(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 Lucene90PostingsReader(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 Lucene90PostingsReader} 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 start of the skip list, relative to docStartFP, if there are more than
* {@link ForUtil#BLOCK_SIZE} docs; otherwise -1
*/
public long skipOffset;
/**
* file offset for the last position in the last block, if there are more than {@link
* ForUtil#BLOCK_SIZE} positions; otherwise -1
*/
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() {
skipOffset = -1;
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;
skipOffset = other.skipOffset;
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,548 @@
/*
* 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.lucene90;
import static org.apache.lucene.codecs.lucene90.ForUtil.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.DOC_CODEC;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.MAX_SKIP_LEVELS;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.PAY_CODEC;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.POS_CODEC;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.TERMS_CODEC;
import static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.VERSION_CURRENT;
import java.io.IOException;
import java.nio.ByteOrder;
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.lucene90.Lucene90PostingsFormat.IntBlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
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.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;
/**
* Concrete class that writes docId(maybe frq,pos,offset,payloads) list with postings format.
*
* <p>Postings list for each term will be stored separately.
*
* @see Lucene90SkipWriter for details about skipping setting and postings layout.
* @lucene.experimental
*/
public final class Lucene90PostingsWriter extends PushPostingsWriterBase {
IndexOutput docOut;
IndexOutput posOut;
IndexOutput payOut;
static final IntBlockTermState emptyState = new IntBlockTermState();
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 lastBlockDocID;
private long lastBlockPosFP;
private long lastBlockPayFP;
private int lastBlockPosBufferUpto;
private int lastBlockPayloadByteUpto;
private int lastDocID;
private int lastPosition;
private int lastStartOffset;
private int docCount;
private final PForUtil pforUtil;
private final ForDeltaUtil forDeltaUtil;
private final Lucene90SkipWriter skipWriter;
private boolean fieldHasNorms;
private NumericDocValues norms;
private final CompetitiveImpactAccumulator competitiveFreqNormAccumulator =
new CompetitiveImpactAccumulator();
/** Creates a postings writer */
public Lucene90PostingsWriter(SegmentWriteState state) throws IOException {
String docFileName =
IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, Lucene90PostingsFormat.DOC_EXTENSION);
docOut = state.directory.createOutput(docFileName, state.context);
IndexOutput posOut = null;
IndexOutput payOut = null;
boolean success = false;
try {
CodecUtil.writeIndexHeader(
docOut, DOC_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
ByteOrder byteOrder = ByteOrder.nativeOrder();
if (byteOrder == ByteOrder.BIG_ENDIAN) {
docOut.writeByte((byte) 'B');
} else if (byteOrder == ByteOrder.LITTLE_ENDIAN) {
docOut.writeByte((byte) 'L');
} else {
throw new Error();
}
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, Lucene90PostingsFormat.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,
Lucene90PostingsFormat.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(docOut, posOut, payOut);
}
}
docDeltaBuffer = new long[BLOCK_SIZE];
freqBuffer = new long[BLOCK_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
skipWriter =
new Lucene90SkipWriter(
MAX_SKIP_LEVELS, BLOCK_SIZE, state.segmentInfo.maxDoc(), docOut, posOut, payOut);
}
@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);
skipWriter.setField(writePositions, writeOffsets, writePayloads);
lastState = emptyState;
fieldHasNorms = fieldInfo.hasNorms();
}
@Override
public void startTerm(NumericDocValues norms) {
docStartFP = docOut.getFilePointer();
if (writePositions) {
posStartFP = posOut.getFilePointer();
if (writePayloads || writeOffsets) {
payStartFP = payOut.getFilePointer();
}
}
lastDocID = 0;
lastBlockDocID = -1;
skipWriter.resetSkip();
this.norms = norms;
competitiveFreqNormAccumulator.clear();
}
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
// Have collected a block of docs, and get a new doc.
// Should write skip data as well as postings list for
// current block.
if (lastBlockDocID != -1 && docBufferUpto == 0) {
skipWriter.bufferSkip(
lastBlockDocID,
competitiveFreqNormAccumulator,
docCount,
lastBlockPosFP,
lastBlockPayFP,
lastBlockPosBufferUpto,
lastBlockPayloadByteUpto);
competitiveFreqNormAccumulator.clear();
}
final int docDelta = docID - lastDocID;
if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
throw new CorruptIndexException(
"docs out of order (" + docID + " <= " + lastDocID + " )", docOut);
}
docDeltaBuffer[docBufferUpto] = docDelta;
if (writeFreqs) {
freqBuffer[docBufferUpto] = termDocFreq;
}
docBufferUpto++;
docCount++;
if (docBufferUpto == BLOCK_SIZE) {
forDeltaUtil.encodeDeltas(docDeltaBuffer, docOut);
if (writeFreqs) {
pforUtil.encode(freqBuffer, docOut);
}
// NOTE: don't set docBufferUpto back to 0 here;
// finishDoc will do so (because it needs to see that
// the block was filled so it can save skip data)
}
lastDocID = docID;
lastPosition = 0;
lastStartOffset = 0;
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;
}
competitiveFreqNormAccumulator.add(writeFreqs ? termDocFreq : 1, 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 {
// Since we don't know df for current term, we had to buffer
// those skip data for each block, and when a new doc comes,
// write them to skip file.
if (docBufferUpto == BLOCK_SIZE) {
lastBlockDocID = lastDocID;
if (posOut != null) {
if (payOut != null) {
lastBlockPayFP = payOut.getFilePointer();
}
lastBlockPosFP = posOut.getFilePointer();
lastBlockPosBufferUpto = posBufferUpto;
lastBlockPayloadByteUpto = payloadByteUpto;
}
docBufferUpto = 0;
}
}
/** 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];
} else {
singletonDocID = -1;
// vInt encode the remaining doc deltas and freqs:
for (int i = 0; i < docBufferUpto; i++) {
final int docDelta = (int) docDeltaBuffer[i];
final int freq = (int) freqBuffer[i];
if (!writeFreqs) {
docOut.writeVInt(docDelta);
} else if (freq == 1) {
docOut.writeVInt((docDelta << 1) | 1);
} else {
docOut.writeVInt(docDelta << 1);
docOut.writeVInt(freq);
}
}
}
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) {
// 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;
}
long skipOffset;
if (docCount > BLOCK_SIZE) {
skipOffset = skipWriter.writeSkip(docOut) - docStartFP;
} else {
skipOffset = -1;
}
state.docStartFP = docStartFP;
state.posStartFP = posStartFP;
state.payStartFP = payStartFP;
state.singletonDocID = singletonDocID;
state.skipOffset = skipOffset;
state.lastPosBlockOffset = lastPosBlockOffset;
docBufferUpto = 0;
posBufferUpto = 0;
lastDocID = 0;
docCount = 0;
}
@Override
public void encodeTerm(
DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute)
throws IOException {
IntBlockTermState state = (IntBlockTermState) _state;
if (absolute) {
lastState = emptyState;
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);
}
}
if (state.skipOffset != -1) {
out.writeVLong(state.skipOffset);
}
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);
}
success = true;
} finally {
if (success) {
IOUtils.close(docOut, posOut, payOut);
} else {
IOUtils.closeWhileHandlingException(docOut, posOut, payOut);
}
docOut = posOut = payOut = null;
}
}
}

View File

@ -0,0 +1,159 @@
/*
* 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.lucene90;
import java.io.IOException;
import java.util.AbstractList;
import java.util.Arrays;
import java.util.List;
import java.util.RandomAccess;
import org.apache.lucene.index.Impact;
import org.apache.lucene.index.Impacts;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
final class Lucene90ScoreSkipReader extends Lucene90SkipReader {
private final byte[][] impactData;
private final int[] impactDataLength;
private final ByteArrayDataInput badi = new ByteArrayDataInput();
private final Impacts impacts;
private int numLevels = 1;
private final MutableImpactList[] perLevelImpacts;
public Lucene90ScoreSkipReader(
IndexInput skipStream,
int maxSkipLevels,
boolean hasPos,
boolean hasOffsets,
boolean hasPayloads) {
super(skipStream, maxSkipLevels, hasPos, hasOffsets, hasPayloads);
this.impactData = new byte[maxSkipLevels][];
Arrays.fill(impactData, new byte[0]);
this.impactDataLength = new int[maxSkipLevels];
this.perLevelImpacts = new MutableImpactList[maxSkipLevels];
for (int i = 0; i < perLevelImpacts.length; ++i) {
perLevelImpacts[i] = new MutableImpactList();
}
impacts =
new Impacts() {
@Override
public int numLevels() {
return numLevels;
}
@Override
public int getDocIdUpTo(int level) {
return skipDoc[level];
}
@Override
public List<Impact> getImpacts(int level) {
assert level < numLevels;
if (impactDataLength[level] > 0) {
badi.reset(impactData[level], 0, impactDataLength[level]);
perLevelImpacts[level] = readImpacts(badi, perLevelImpacts[level]);
impactDataLength[level] = 0;
}
return perLevelImpacts[level];
}
};
}
@Override
public int skipTo(int target) throws IOException {
int result = super.skipTo(target);
if (numberOfSkipLevels > 0) {
numLevels = numberOfSkipLevels;
} else {
// End of postings don't have skip data anymore, so we fill with dummy data
// like SlowImpactsEnum.
numLevels = 1;
perLevelImpacts[0].length = 1;
perLevelImpacts[0].impacts[0].freq = Integer.MAX_VALUE;
perLevelImpacts[0].impacts[0].norm = 1L;
impactDataLength[0] = 0;
}
return result;
}
Impacts getImpacts() {
return impacts;
}
@Override
protected void readImpacts(int level, IndexInput skipStream) throws IOException {
int length = skipStream.readVInt();
if (impactData[level].length < length) {
impactData[level] = new byte[ArrayUtil.oversize(length, Byte.BYTES)];
}
skipStream.readBytes(impactData[level], 0, length);
impactDataLength[level] = length;
}
static MutableImpactList readImpacts(ByteArrayDataInput in, MutableImpactList reuse) {
int maxNumImpacts = in.length(); // at most one impact per byte
if (reuse.impacts.length < maxNumImpacts) {
int oldLength = reuse.impacts.length;
reuse.impacts = ArrayUtil.grow(reuse.impacts, maxNumImpacts);
for (int i = oldLength; i < reuse.impacts.length; ++i) {
reuse.impacts[i] = new Impact(Integer.MAX_VALUE, 1L);
}
}
int freq = 0;
long norm = 0;
int length = 0;
while (in.getPosition() < in.length()) {
int freqDelta = in.readVInt();
if ((freqDelta & 0x01) != 0) {
freq += 1 + (freqDelta >>> 1);
try {
norm += 1 + in.readZLong();
} catch (IOException e) {
throw new RuntimeException(e); // cannot happen on a BADI
}
} else {
freq += 1 + (freqDelta >>> 1);
norm++;
}
Impact impact = reuse.impacts[length];
impact.freq = freq;
impact.norm = norm;
length++;
}
reuse.length = length;
return reuse;
}
static class MutableImpactList extends AbstractList<Impact> implements RandomAccess {
int length = 1;
Impact[] impacts = new Impact[] {new Impact(Integer.MAX_VALUE, 1L)};
@Override
public Impact get(int index) {
return impacts[index];
}
@Override
public int size() {
return length;
}
}
}

View File

@ -0,0 +1,206 @@
/*
* 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.lucene90;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.MultiLevelSkipListReader;
import org.apache.lucene.store.IndexInput;
/**
* Implements the skip list reader for block postings format that stores positions and payloads.
*
* <p>Although this skipper uses MultiLevelSkipListReader as an interface, its definition of skip
* position will be a little different.
*
* <p>For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6,
*
* <pre>
* 0 1 2 3 4 5
* d d d d d d (posting list)
* ^ ^ (skip point in MultiLeveSkipWriter)
* ^ (skip point in Lucene90SkipWriter)
* </pre>
*
* <p>In this case, MultiLevelSkipListReader will use the last document as a skip point, while
* Lucene90SkipReader should assume no skip point will comes.
*
* <p>If we use the interface directly in Lucene90SkipReader, it may silly try to read another skip
* data after the only skip point is loaded.
*
* <p>To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId, and
* numSkipped+blockSize== df, the MultiLevelSkipListReader will assume the skip list isn't exhausted
* yet, and try to load a non-existed skip point
*
* <p>Therefore, we'll trim df before passing it to the interface. see trim(int)
*/
class Lucene90SkipReader extends MultiLevelSkipListReader {
private long docPointer[];
private long posPointer[];
private long payPointer[];
private int posBufferUpto[];
private int payloadByteUpto[];
private long lastPosPointer;
private long lastPayPointer;
private int lastPayloadByteUpto;
private long lastDocPointer;
private int lastPosBufferUpto;
public Lucene90SkipReader(
IndexInput skipStream,
int maxSkipLevels,
boolean hasPos,
boolean hasOffsets,
boolean hasPayloads) {
super(skipStream, maxSkipLevels, ForUtil.BLOCK_SIZE, 8);
docPointer = new long[maxSkipLevels];
if (hasPos) {
posPointer = new long[maxSkipLevels];
posBufferUpto = new int[maxSkipLevels];
if (hasPayloads) {
payloadByteUpto = new int[maxSkipLevels];
} else {
payloadByteUpto = null;
}
if (hasOffsets || hasPayloads) {
payPointer = new long[maxSkipLevels];
} else {
payPointer = null;
}
} else {
posPointer = null;
}
}
/**
* Trim original docFreq to tell skipReader read proper number of skip points.
*
* <p>Since our definition in Lucene90Skip* is a little different from MultiLevelSkip* This
* trimmed docFreq will prevent skipReader from: 1. silly reading a non-existed skip point after
* the last block boundary 2. moving into the vInt block
*/
protected int trim(int df) {
return df % ForUtil.BLOCK_SIZE == 0 ? df - 1 : df;
}
public void init(
long skipPointer, long docBasePointer, long posBasePointer, long payBasePointer, int df)
throws IOException {
super.init(skipPointer, trim(df));
lastDocPointer = docBasePointer;
lastPosPointer = posBasePointer;
lastPayPointer = payBasePointer;
Arrays.fill(docPointer, docBasePointer);
if (posPointer != null) {
Arrays.fill(posPointer, posBasePointer);
if (payPointer != null) {
Arrays.fill(payPointer, payBasePointer);
}
} else {
assert posBasePointer == 0;
}
}
/**
* Returns the doc pointer of the doc to which the last call of {@link
* MultiLevelSkipListReader#skipTo(int)} has skipped.
*/
public long getDocPointer() {
return lastDocPointer;
}
public long getPosPointer() {
return lastPosPointer;
}
public int getPosBufferUpto() {
return lastPosBufferUpto;
}
public long getPayPointer() {
return lastPayPointer;
}
public int getPayloadByteUpto() {
return lastPayloadByteUpto;
}
public int getNextSkipDoc() {
return skipDoc[0];
}
@Override
protected void seekChild(int level) throws IOException {
super.seekChild(level);
docPointer[level] = lastDocPointer;
if (posPointer != null) {
posPointer[level] = lastPosPointer;
posBufferUpto[level] = lastPosBufferUpto;
if (payloadByteUpto != null) {
payloadByteUpto[level] = lastPayloadByteUpto;
}
if (payPointer != null) {
payPointer[level] = lastPayPointer;
}
}
}
@Override
protected void setLastSkipData(int level) {
super.setLastSkipData(level);
lastDocPointer = docPointer[level];
if (posPointer != null) {
lastPosPointer = posPointer[level];
lastPosBufferUpto = posBufferUpto[level];
if (payPointer != null) {
lastPayPointer = payPointer[level];
}
if (payloadByteUpto != null) {
lastPayloadByteUpto = payloadByteUpto[level];
}
}
}
@Override
protected int readSkipData(int level, IndexInput skipStream) throws IOException {
int delta = skipStream.readVInt();
docPointer[level] += skipStream.readVLong();
if (posPointer != null) {
posPointer[level] += skipStream.readVLong();
posBufferUpto[level] = skipStream.readVInt();
if (payloadByteUpto != null) {
payloadByteUpto[level] = skipStream.readVInt();
}
if (payPointer != null) {
payPointer[level] += skipStream.readVLong();
}
}
readImpacts(level, skipStream);
return delta;
}
// The default impl skips impacts
protected void readImpacts(int level, IndexInput skipStream) throws IOException {
skipStream.skipBytes(skipStream.readVInt());
}
}

View File

@ -0,0 +1,237 @@
/*
* 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.lucene90;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.index.Impact;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
/**
* Write skip lists with multiple levels, and support skip within block ints.
*
* <p>Assume that docFreq = 28, skipInterval = blockSize = 12
*
* <pre>
* | block#0 | | block#1 | |vInts|
* d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list)
* ^ ^ (level 0 skip point)
* </pre>
*
* <p>Note that skipWriter will ignore first document in block#0, since it is useless as a skip
* point. Also, we'll never skip into the vInts block, only record skip data at the start its start
* point(if it exist).
*
* <p>For each skip point, we will record: 1. docID in former position, i.e. for position 12, record
* docID[11], etc. 2. its related file points(position, payload), 3. related numbers or
* uptos(position, payload). 4. start offset.
*/
final class Lucene90SkipWriter extends MultiLevelSkipListWriter {
private int[] lastSkipDoc;
private long[] lastSkipDocPointer;
private long[] lastSkipPosPointer;
private long[] lastSkipPayPointer;
private int[] lastPayloadByteUpto;
private final IndexOutput docOut;
private final IndexOutput posOut;
private final IndexOutput payOut;
private int curDoc;
private long curDocPointer;
private long curPosPointer;
private long curPayPointer;
private int curPosBufferUpto;
private int curPayloadByteUpto;
private CompetitiveImpactAccumulator[] curCompetitiveFreqNorms;
private boolean fieldHasPositions;
private boolean fieldHasOffsets;
private boolean fieldHasPayloads;
public Lucene90SkipWriter(
int maxSkipLevels,
int blockSize,
int docCount,
IndexOutput docOut,
IndexOutput posOut,
IndexOutput payOut) {
super(blockSize, 8, maxSkipLevels, docCount);
this.docOut = docOut;
this.posOut = posOut;
this.payOut = payOut;
lastSkipDoc = new int[maxSkipLevels];
lastSkipDocPointer = new long[maxSkipLevels];
if (posOut != null) {
lastSkipPosPointer = new long[maxSkipLevels];
if (payOut != null) {
lastSkipPayPointer = new long[maxSkipLevels];
}
lastPayloadByteUpto = new int[maxSkipLevels];
}
curCompetitiveFreqNorms = new CompetitiveImpactAccumulator[maxSkipLevels];
for (int i = 0; i < maxSkipLevels; ++i) {
curCompetitiveFreqNorms[i] = new CompetitiveImpactAccumulator();
}
}
public void setField(
boolean fieldHasPositions, boolean fieldHasOffsets, boolean fieldHasPayloads) {
this.fieldHasPositions = fieldHasPositions;
this.fieldHasOffsets = fieldHasOffsets;
this.fieldHasPayloads = fieldHasPayloads;
}
// tricky: we only skip data for blocks (terms with more than 128 docs), but re-init'ing the
// skipper
// is pretty slow for rare terms in large segments as we have to fill O(log #docs in segment) of
// junk.
// this is the vast majority of terms (worst case: ID field or similar). so in resetSkip() we
// save
// away the previous pointers, and lazy-init only if we need to buffer skip data for the term.
private boolean initialized;
long lastDocFP;
long lastPosFP;
long lastPayFP;
@Override
public void resetSkip() {
lastDocFP = docOut.getFilePointer();
if (fieldHasPositions) {
lastPosFP = posOut.getFilePointer();
if (fieldHasOffsets || fieldHasPayloads) {
lastPayFP = payOut.getFilePointer();
}
}
if (initialized) {
for (CompetitiveImpactAccumulator acc : curCompetitiveFreqNorms) {
acc.clear();
}
}
initialized = false;
}
private void initSkip() {
if (!initialized) {
super.resetSkip();
Arrays.fill(lastSkipDoc, 0);
Arrays.fill(lastSkipDocPointer, lastDocFP);
if (fieldHasPositions) {
Arrays.fill(lastSkipPosPointer, lastPosFP);
if (fieldHasPayloads) {
Arrays.fill(lastPayloadByteUpto, 0);
}
if (fieldHasOffsets || fieldHasPayloads) {
Arrays.fill(lastSkipPayPointer, lastPayFP);
}
}
// sets of competitive freq,norm pairs should be empty at this point
assert Arrays.stream(curCompetitiveFreqNorms)
.map(CompetitiveImpactAccumulator::getCompetitiveFreqNormPairs)
.mapToInt(Collection::size)
.sum()
== 0;
initialized = true;
}
}
/** Sets the values for the current skip data. */
public void bufferSkip(
int doc,
CompetitiveImpactAccumulator competitiveFreqNorms,
int numDocs,
long posFP,
long payFP,
int posBufferUpto,
int payloadByteUpto)
throws IOException {
initSkip();
this.curDoc = doc;
this.curDocPointer = docOut.getFilePointer();
this.curPosPointer = posFP;
this.curPayPointer = payFP;
this.curPosBufferUpto = posBufferUpto;
this.curPayloadByteUpto = payloadByteUpto;
this.curCompetitiveFreqNorms[0].addAll(competitiveFreqNorms);
bufferSkip(numDocs);
}
private final ByteBuffersDataOutput freqNormOut = ByteBuffersDataOutput.newResettableInstance();
@Override
protected void writeSkipData(int level, DataOutput skipBuffer) throws IOException {
int delta = curDoc - lastSkipDoc[level];
skipBuffer.writeVInt(delta);
lastSkipDoc[level] = curDoc;
skipBuffer.writeVLong(curDocPointer - lastSkipDocPointer[level]);
lastSkipDocPointer[level] = curDocPointer;
if (fieldHasPositions) {
skipBuffer.writeVLong(curPosPointer - lastSkipPosPointer[level]);
lastSkipPosPointer[level] = curPosPointer;
skipBuffer.writeVInt(curPosBufferUpto);
if (fieldHasPayloads) {
skipBuffer.writeVInt(curPayloadByteUpto);
}
if (fieldHasOffsets || fieldHasPayloads) {
skipBuffer.writeVLong(curPayPointer - lastSkipPayPointer[level]);
lastSkipPayPointer[level] = curPayPointer;
}
}
CompetitiveImpactAccumulator competitiveFreqNorms = curCompetitiveFreqNorms[level];
assert competitiveFreqNorms.getCompetitiveFreqNormPairs().size() > 0;
if (level + 1 < numberOfSkipLevels) {
curCompetitiveFreqNorms[level + 1].addAll(competitiveFreqNorms);
}
writeImpacts(competitiveFreqNorms, freqNormOut);
skipBuffer.writeVInt(Math.toIntExact(freqNormOut.size()));
freqNormOut.copyTo(skipBuffer);
freqNormOut.reset();
competitiveFreqNorms.clear();
}
static void writeImpacts(CompetitiveImpactAccumulator acc, DataOutput out) throws IOException {
Collection<Impact> impacts = acc.getCompetitiveFreqNormPairs();
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;
}
}
}

View File

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

View File

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

View File

@ -0,0 +1,227 @@
/*
* 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.lucene90.blocktree;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.OffHeapFSTStore;
/**
* BlockTree's implementation of {@link Terms}.
*
* @lucene.internal
*/
public final class FieldReader extends Terms implements Accountable {
// private final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(FieldReader.class)
+ 3 * RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final long rootBlockFP;
final BytesRef rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final Lucene90BlockTreeTermsReader parent;
final FST<BytesRef> index;
// private boolean DEBUG;
FieldReader(
Lucene90BlockTreeTermsReader parent,
FieldInfo fieldInfo,
long numTerms,
BytesRef rootCode,
long sumTotalTermFreq,
long sumDocFreq,
int docCount,
long indexStartFP,
IndexInput metaIn,
IndexInput indexIn,
BytesRef minTerm,
BytesRef maxTerm)
throws IOException {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
// DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
this.parent = parent;
this.numTerms = numTerms;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.rootCode = rootCode;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// if (DEBUG) {
// System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode="
// + rootCode + " divisor=" + indexDivisor);
// }
rootBlockFP =
(new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong()
>>> Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
// Initialize FST always off-heap.
final IndexInput clone = indexIn.clone();
clone.seek(indexStartFP);
index = new FST<>(metaIn, clone, ByteSequenceOutputs.getSingleton(), new OffHeapFSTStore());
/*
if (false) {
final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
Util.toDot(index, w, false, false);
System.out.println("FST INDEX: SAVED to " + dotFileName);
w.close();
}
*/
}
@Override
public BytesRef getMin() throws IOException {
if (minTerm == null) {
// Older index that didn't store min/maxTerm
return super.getMin();
} else {
return minTerm;
}
}
@Override
public BytesRef getMax() throws IOException {
if (maxTerm == null) {
// Older index that didn't store min/maxTerm
return super.getMax();
} else {
return maxTerm;
}
}
/** For debugging -- used by CheckIndex too */
@Override
public Stats getStats() throws IOException {
return new SegmentTermsEnum(this).computeBlockStats();
}
@Override
public boolean hasFreqs() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
}
@Override
public boolean hasOffsets() {
return fieldInfo
.getIndexOptions()
.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS)
>= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldInfo.hasPayloads();
}
@Override
public TermsEnum iterator() throws IOException {
return new SegmentTermsEnum(this);
}
@Override
public long size() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public long getSumDocFreq() {
return sumDocFreq;
}
@Override
public int getDocCount() {
return docCount;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
// if (DEBUG) System.out.println(" FieldReader.intersect startTerm=" +
// BlockTreeTermsWriter.brToString(startTerm));
// System.out.println("intersect: " + compiled.type + " a=" + compiled.automaton);
// TODO: we could push "it's a range" or "it's a prefix" down into IntersectTermsEnum?
// can we optimize knowing that...?
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
return new IntersectTermsEnum(
this, compiled.automaton, compiled.runAutomaton, compiled.commonSuffixRef, startTerm);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + ((index != null) ? index.ramBytesUsed() : 0);
}
@Override
public Collection<Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singleton(Accountables.namedAccountable("term index", index));
}
}
@Override
public String toString() {
return "BlockTreeTerms(seg="
+ parent.segment
+ " terms="
+ numTerms
+ ",postings="
+ sumDocFreq
+ ",positions="
+ sumTotalTermFreq
+ ",docs="
+ docCount
+ ")";
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.codecs.lucene90.blocktree;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.BaseTermsEnum;

View File

@ -0,0 +1,329 @@
/*
* 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.lucene90.blocktree;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.FST;
// TODO: can we share this with the frame in STE?
final class IntersectTermsEnumFrame {
final int ord;
long fp;
long fpOrig;
long fpEnd;
long lastSubFP;
// private static boolean DEBUG = IntersectTermsEnum.DEBUG;
// State in automaton
int state;
// State just before the last label
int lastState;
int metaDataUpto;
byte[] suffixBytes = new byte[128];
final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
byte[] suffixLengthBytes;
final ByteArrayDataInput suffixLengthsReader;
byte[] statBytes = new byte[64];
int statsSingletonRunLength = 0;
final ByteArrayDataInput statsReader = new ByteArrayDataInput();
byte[] floorData = new byte[32];
final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
// Length of prefix shared by all terms in this block
int prefix;
// Number of entries (term or sub-block) in this block
int entCount;
// Which term we will next read
int nextEnt;
// True if this block is either not a floor block,
// or, it's the last sub-block of a floor block
boolean isLastInFloor;
// True if all entries are terms
boolean isLeafBlock;
int numFollowFloorBlocks;
int nextFloorLabel;
final Transition transition = new Transition();
int transitionIndex;
int transitionCount;
FST.Arc<BytesRef> arc;
final BlockTermState termState;
// metadata buffer
byte[] bytes = new byte[32];
final ByteArrayDataInput bytesReader = new ByteArrayDataInput();
// Cumulative output so far
BytesRef outputPrefix;
int startBytePos;
int suffix;
private final IntersectTermsEnum ite;
private final int version;
public IntersectTermsEnumFrame(IntersectTermsEnum ite, int ord) throws IOException {
this.ite = ite;
this.ord = ord;
this.termState = ite.fr.parent.postingsReader.newTermState();
this.termState.totalTermFreq = -1;
this.version = ite.fr.parent.version;
suffixLengthBytes = new byte[32];
suffixLengthsReader = new ByteArrayDataInput();
}
void loadNextFloorBlock() throws IOException {
assert numFollowFloorBlocks > 0 : "nextFloorLabel=" + nextFloorLabel;
do {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
if (numFollowFloorBlocks != 0) {
nextFloorLabel = floorDataReader.readByte() & 0xff;
} else {
nextFloorLabel = 256;
}
} while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min);
load(null);
}
public void setState(int state) {
this.state = state;
transitionIndex = 0;
transitionCount = ite.automaton.getNumTransitions(state);
if (transitionCount != 0) {
ite.automaton.initTransition(state, transition);
ite.automaton.getNextTransition(transition);
} else {
// Must set min to -1 so the "label < min" check never falsely triggers:
transition.min = -1;
// Must set max to -1 so we immediately realize we need to step to the next transition and
// then pop this frame:
transition.max = -1;
}
}
void load(BytesRef frameIndexData) throws IOException {
if (frameIndexData != null) {
floorDataReader.reset(frameIndexData.bytes, frameIndexData.offset, frameIndexData.length);
// Skip first long -- has redundant fp, hasTerms
// flag, isFloor flag
final long code = floorDataReader.readVLong();
if ((code & Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) {
// Floor frame
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
// If current state is not accept, and has transitions, we must process
// first block in case it has empty suffix:
if (ite.runAutomaton.isAccept(state) == false && transitionCount != 0) {
// Maybe skip floor blocks:
assert transitionIndex == 0 : "transitionIndex=" + transitionIndex;
while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min) {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
if (numFollowFloorBlocks != 0) {
nextFloorLabel = floorDataReader.readByte() & 0xff;
} else {
nextFloorLabel = 256;
}
}
}
}
}
ite.in.seek(fp);
int code = ite.in.readVInt();
entCount = code >>> 1;
assert entCount > 0;
isLastInFloor = (code & 1) != 0;
// term suffixes:
final long codeL = ite.in.readVLong();
isLeafBlock = (codeL & 0x04) != 0;
final int numSuffixBytes = (int) (codeL >>> 3);
if (suffixBytes.length < numSuffixBytes) {
suffixBytes = new byte[ArrayUtil.oversize(numSuffixBytes, 1)];
}
final CompressionAlgorithm compressionAlg;
try {
compressionAlg = CompressionAlgorithm.byCode((int) codeL & 0x03);
} catch (IllegalArgumentException e) {
throw new CorruptIndexException(e.getMessage(), ite.in, e);
}
compressionAlg.read(ite.in, suffixBytes, numSuffixBytes);
suffixesReader.reset(suffixBytes, 0, numSuffixBytes);
int numSuffixLengthBytes = ite.in.readVInt();
final boolean allEqual = (numSuffixLengthBytes & 0x01) != 0;
numSuffixLengthBytes >>>= 1;
if (suffixLengthBytes.length < numSuffixLengthBytes) {
suffixLengthBytes = new byte[ArrayUtil.oversize(numSuffixLengthBytes, 1)];
}
if (allEqual) {
Arrays.fill(suffixLengthBytes, 0, numSuffixLengthBytes, ite.in.readByte());
} else {
ite.in.readBytes(suffixLengthBytes, 0, numSuffixLengthBytes);
}
suffixLengthsReader.reset(suffixLengthBytes, 0, numSuffixLengthBytes);
// stats
int numBytes = ite.in.readVInt();
if (statBytes.length < numBytes) {
statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ite.in.readBytes(statBytes, 0, numBytes);
statsReader.reset(statBytes, 0, numBytes);
statsSingletonRunLength = 0;
metaDataUpto = 0;
termState.termBlockOrd = 0;
nextEnt = 0;
// metadata
numBytes = ite.in.readVInt();
if (bytes.length < numBytes) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ite.in.readBytes(bytes, 0, numBytes);
bytesReader.reset(bytes, 0, numBytes);
if (!isLastInFloor) {
// Sub-blocks of a single floor block are always
// written one after another -- tail recurse:
fpEnd = ite.in.getFilePointer();
}
}
// TODO: maybe add scanToLabel; should give perf boost
// Decodes next entry; returns true if it's a sub-block
public boolean next() {
if (isLeafBlock) {
nextLeaf();
return false;
} else {
return nextNonLeaf();
}
}
public void nextLeaf() {
assert nextEnt != -1 && nextEnt < entCount
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
suffix = suffixLengthsReader.readVInt();
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
}
public boolean nextNonLeaf() {
assert nextEnt != -1 && nextEnt < entCount
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffix = code >>> 1;
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
if ((code & 1) == 0) {
// A normal term
termState.termBlockOrd++;
return false;
} else {
// A sub-block; make sub-FP absolute:
lastSubFP = fp - suffixLengthsReader.readVLong();
return true;
}
}
public int getTermBlockOrd() {
return isLeafBlock ? nextEnt : termState.termBlockOrd;
}
public void decodeMetaData() throws IOException {
// lazily catch up on metadata decode:
final int limit = getTermBlockOrd();
boolean absolute = metaDataUpto == 0;
assert limit > 0;
// TODO: better API would be "jump straight to term=N"???
while (metaDataUpto < limit) {
// TODO: we could make "tiers" of metadata, ie,
// decode docFreq/totalTF but don't decode postings
// metadata; this way caller could get
// docFreq/totalTF w/o paying decode cost for
// postings
// TODO: if docFreq were bulk decoded we could
// just skipN here:
// stats
if (statsSingletonRunLength > 0) {
termState.docFreq = 1;
termState.totalTermFreq = 1;
statsSingletonRunLength--;
} else {
int token = statsReader.readVInt();
if ((token & 1) == 1) {
termState.docFreq = 1;
termState.totalTermFreq = 1;
statsSingletonRunLength = token >>> 1;
} else {
termState.docFreq = token >>> 1;
if (ite.fr.fieldInfo.getIndexOptions() == IndexOptions.DOCS) {
termState.totalTermFreq = termState.docFreq;
} else {
termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
}
}
}
// metadata
ite.fr.parent.postingsReader.decodeTerm(bytesReader, ite.fr.fieldInfo, termState, absolute);
metaDataUpto++;
absolute = false;
}
termState.termBlockOrd = metaDataUpto;
}
}

View File

@ -0,0 +1,367 @@
/*
* 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.lucene90.blocktree;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.Outputs;
/**
* A block-based terms index and dictionary that assigns terms to variable length blocks according
* to how they share prefixes. The terms index is a prefix trie whose leaves are term blocks. The
* advantage of this approach is that seekExact is often able to determine a term cannot exist
* without doing any IO, and intersection with Automata is very fast. Note that this terms
* dictionary has its own fixed terms index (ie, it does not support a pluggable terms index
* implementation).
*
* <p><b>NOTE</b>: this terms dictionary supports min/maxItemsPerBlock during indexing to control
* how much memory the terms index uses.
*
* <p>The data structure used by this implementation is very similar to a burst trie
* (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499), but with added logic to break
* up too-large blocks of all terms sharing a given prefix into smaller ones.
*
* <p>Use {@link org.apache.lucene.index.CheckIndex} with the <code>-verbose</code> option to see
* summary statistics on the blocks in the dictionary.
*
* <p>See {@link Lucene90BlockTreeTermsWriter}.
*
* @lucene.experimental
*/
public final class Lucene90BlockTreeTermsReader extends FieldsProducer {
static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton();
static final BytesRef NO_OUTPUT = FST_OUTPUTS.getNoOutput();
static final int OUTPUT_FLAGS_NUM_BITS = 2;
static final int OUTPUT_FLAGS_MASK = 0x3;
static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
/** Extension of terms file */
static final String TERMS_EXTENSION = "tim";
static final String TERMS_CODEC_NAME = "BlockTreeTermsDict";
/** Initial terms format. */
public static final int VERSION_START = 0;
/** Current terms format. */
public static final int VERSION_CURRENT = VERSION_START;
/** Extension of terms index file */
static final String TERMS_INDEX_EXTENSION = "tip";
static final String TERMS_INDEX_CODEC_NAME = "BlockTreeTermsIndex";
/** Extension of terms meta file */
static final String TERMS_META_EXTENSION = "tmd";
static final String TERMS_META_CODEC_NAME = "BlockTreeTermsMeta";
// Open input to the main terms dict file (_X.tib)
final IndexInput termsIn;
// Open input to the terms index file (_X.tip)
final IndexInput indexIn;
// private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
// Reads the terms dict entries, to gather state to
// produce DocsEnum on demand
final PostingsReaderBase postingsReader;
private final Map<String, FieldReader> fieldMap;
private final List<String> fieldList;
final String segment;
final int version;
/** Sole constructor. */
public Lucene90BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state)
throws IOException {
boolean success = false;
this.postingsReader = postingsReader;
this.segment = state.segmentInfo.name;
try {
String termsName =
IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_EXTENSION);
termsIn = state.directory.openInput(termsName, state.context);
version =
CodecUtil.checkIndexHeader(
termsIn,
TERMS_CODEC_NAME,
VERSION_START,
VERSION_CURRENT,
state.segmentInfo.getId(),
state.segmentSuffix);
String indexName =
IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_INDEX_EXTENSION);
indexIn = state.directory.openInput(indexName, state.context);
CodecUtil.checkIndexHeader(
indexIn,
TERMS_INDEX_CODEC_NAME,
version,
version,
state.segmentInfo.getId(),
state.segmentSuffix);
// Read per-field details
String metaName =
IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_META_EXTENSION);
Map<String, FieldReader> fieldMap = null;
Throwable priorE = null;
long indexLength = -1, termsLength = -1;
try (ChecksumIndexInput metaIn = state.directory.openChecksumInput(metaName, state.context)) {
try {
final IndexInput indexMetaIn, termsMetaIn;
CodecUtil.checkIndexHeader(
metaIn,
TERMS_META_CODEC_NAME,
version,
version,
state.segmentInfo.getId(),
state.segmentSuffix);
indexMetaIn = termsMetaIn = metaIn;
postingsReader.init(metaIn, state);
final int numFields = termsMetaIn.readVInt();
if (numFields < 0) {
throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
}
fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
for (int i = 0; i < numFields; ++i) {
final int field = termsMetaIn.readVInt();
final long numTerms = termsMetaIn.readVLong();
if (numTerms <= 0) {
throw new CorruptIndexException(
"Illegal numTerms for field number: " + field, termsMetaIn);
}
final BytesRef rootCode = readBytesRef(termsMetaIn);
final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
throw new CorruptIndexException("invalid field number: " + field, termsMetaIn);
}
final long sumTotalTermFreq = termsMetaIn.readVLong();
// when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is
// written.
final long sumDocFreq =
fieldInfo.getIndexOptions() == IndexOptions.DOCS
? sumTotalTermFreq
: termsMetaIn.readVLong();
final int docCount = termsMetaIn.readVInt();
BytesRef minTerm = readBytesRef(termsMetaIn);
BytesRef maxTerm = readBytesRef(termsMetaIn);
if (docCount < 0
|| docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
throw new CorruptIndexException(
"invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(),
termsMetaIn);
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException(
"invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, termsMetaIn);
}
if (sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException(
"invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq,
termsMetaIn);
}
final long indexStartFP = indexMetaIn.readVLong();
FieldReader previous =
fieldMap.put(
fieldInfo.name,
new FieldReader(
this,
fieldInfo,
numTerms,
rootCode,
sumTotalTermFreq,
sumDocFreq,
docCount,
indexStartFP,
indexMetaIn,
indexIn,
minTerm,
maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, termsMetaIn);
}
}
indexLength = metaIn.readLong();
termsLength = metaIn.readLong();
} catch (Throwable exception) {
priorE = exception;
} finally {
if (metaIn != null) {
CodecUtil.checkFooter(metaIn, priorE);
} else if (priorE != null) {
IOUtils.rethrowAlways(priorE);
}
}
}
// At this point the checksum of the meta file has been verified so the lengths are likely
// correct
CodecUtil.retrieveChecksum(indexIn, indexLength);
CodecUtil.retrieveChecksum(termsIn, termsLength);
List<String> fieldList = new ArrayList<>(fieldMap.keySet());
fieldList.sort(null);
this.fieldMap = fieldMap;
this.fieldList = Collections.unmodifiableList(fieldList);
success = true;
} finally {
if (!success) {
// this.close() will close in:
IOUtils.closeWhileHandlingException(this);
}
}
}
private static BytesRef readBytesRef(IndexInput in) throws IOException {
int numBytes = in.readVInt();
if (numBytes < 0) {
throw new CorruptIndexException("invalid bytes length: " + numBytes, in);
}
BytesRef bytes = new BytesRef();
bytes.length = numBytes;
bytes.bytes = new byte[numBytes];
in.readBytes(bytes.bytes, 0, numBytes);
return bytes;
}
/** Seek {@code input} to the directory offset. */
private static void seekDir(IndexInput input) throws IOException {
input.seek(input.length() - CodecUtil.footerLength() - 8);
long offset = input.readLong();
input.seek(offset);
}
// for debugging
// private static String toHex(int v) {
// return "0x" + Integer.toHexString(v);
// }
@Override
public void close() throws IOException {
try {
IOUtils.close(indexIn, termsIn, postingsReader);
} finally {
// Clear so refs to terms index is GCable even if
// app hangs onto us:
fieldMap.clear();
}
}
@Override
public Iterator<String> iterator() {
return fieldList.iterator();
}
@Override
public Terms terms(String field) throws IOException {
assert field != null;
return fieldMap.get(field);
}
@Override
public int size() {
return fieldMap.size();
}
// for debugging
String brToString(BytesRef b) {
if (b == null) {
return "null";
} else {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
// If BytesRef isn't actually UTF8, or it's eg a
// prefix of UTF8 that ends mid-unicode-char, we
// fallback to hex:
return b.toString();
}
}
}
@Override
public long ramBytesUsed() {
long sizeInBytes = postingsReader.ramBytesUsed();
for (FieldReader reader : fieldMap.values()) {
sizeInBytes += reader.ramBytesUsed();
}
return sizeInBytes;
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources =
new ArrayList<>(Accountables.namedAccountables("field", fieldMap));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
// terms index
CodecUtil.checksumEntireFile(indexIn);
// term dictionary
CodecUtil.checksumEntireFile(termsIn);
// postings
postingsReader.checkIntegrity();
}
@Override
public String toString() {
return getClass().getSimpleName()
+ "(fields="
+ fieldMap.size()
+ ",delegate="
+ postingsReader
+ ")";
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.codecs.lucene90.blocktree;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -192,20 +192,20 @@ import org.apache.lucene.util.fst.Util;
* byte of each sub-block, and its file pointer. * byte of each sub-block, and its file pointer.
* </ul> * </ul>
* *
* @see BlockTreeTermsReader * @see Lucene90BlockTreeTermsReader
* @lucene.experimental * @lucene.experimental
*/ */
public final class BlockTreeTermsWriter extends FieldsConsumer { public final class Lucene90BlockTreeTermsWriter extends FieldsConsumer {
/** /**
* Suggested default value for the {@code minItemsInBlock} parameter to {@link * Suggested default value for the {@code minItemsInBlock} parameter to {@link
* #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. * #Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}.
*/ */
public static final int DEFAULT_MIN_BLOCK_SIZE = 25; public static final int DEFAULT_MIN_BLOCK_SIZE = 25;
/** /**
* Suggested default value for the {@code maxItemsInBlock} parameter to {@link * Suggested default value for the {@code maxItemsInBlock} parameter to {@link
* #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. * #Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}.
*/ */
public static final int DEFAULT_MAX_BLOCK_SIZE = 48; public static final int DEFAULT_MAX_BLOCK_SIZE = 48;
@ -231,7 +231,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
* minItemsPerBlock and maxItemsPerBlock, though in some cases the blocks may be smaller than the * minItemsPerBlock and maxItemsPerBlock, though in some cases the blocks may be smaller than the
* min. * min.
*/ */
public BlockTreeTermsWriter( public Lucene90BlockTreeTermsWriter(
SegmentWriteState state, SegmentWriteState state,
PostingsWriterBase postingsWriter, PostingsWriterBase postingsWriter,
int minItemsInBlock, int minItemsInBlock,
@ -248,15 +248,17 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
final String termsName = final String termsName =
IndexFileNames.segmentFileName( IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_EXTENSION); state.segmentInfo.name,
state.segmentSuffix,
Lucene90BlockTreeTermsReader.TERMS_EXTENSION);
termsOut = state.directory.createOutput(termsName, state.context); termsOut = state.directory.createOutput(termsName, state.context);
boolean success = false; boolean success = false;
IndexOutput metaOut = null, indexOut = null; IndexOutput metaOut = null, indexOut = null;
try { try {
CodecUtil.writeIndexHeader( CodecUtil.writeIndexHeader(
termsOut, termsOut,
BlockTreeTermsReader.TERMS_CODEC_NAME, Lucene90BlockTreeTermsReader.TERMS_CODEC_NAME,
BlockTreeTermsReader.VERSION_CURRENT, Lucene90BlockTreeTermsReader.VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentInfo.getId(),
state.segmentSuffix); state.segmentSuffix);
@ -264,12 +266,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
IndexFileNames.segmentFileName( IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentInfo.name,
state.segmentSuffix, state.segmentSuffix,
BlockTreeTermsReader.TERMS_INDEX_EXTENSION); Lucene90BlockTreeTermsReader.TERMS_INDEX_EXTENSION);
indexOut = state.directory.createOutput(indexName, state.context); indexOut = state.directory.createOutput(indexName, state.context);
CodecUtil.writeIndexHeader( CodecUtil.writeIndexHeader(
indexOut, indexOut,
BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, Lucene90BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME,
BlockTreeTermsReader.VERSION_CURRENT, Lucene90BlockTreeTermsReader.VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentInfo.getId(),
state.segmentSuffix); state.segmentSuffix);
// segment = state.segmentInfo.name; // segment = state.segmentInfo.name;
@ -278,12 +280,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
IndexFileNames.segmentFileName( IndexFileNames.segmentFileName(
state.segmentInfo.name, state.segmentInfo.name,
state.segmentSuffix, state.segmentSuffix,
BlockTreeTermsReader.TERMS_META_EXTENSION); Lucene90BlockTreeTermsReader.TERMS_META_EXTENSION);
metaOut = state.directory.createOutput(metaName, state.context); metaOut = state.directory.createOutput(metaName, state.context);
CodecUtil.writeIndexHeader( CodecUtil.writeIndexHeader(
metaOut, metaOut,
BlockTreeTermsReader.TERMS_META_CODEC_NAME, Lucene90BlockTreeTermsReader.TERMS_META_CODEC_NAME,
BlockTreeTermsReader.VERSION_CURRENT, Lucene90BlockTreeTermsReader.VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentInfo.getId(),
state.segmentSuffix); state.segmentSuffix);
@ -359,8 +361,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) { static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
assert fp < (1L << 62); assert fp < (1L << 62);
return (fp << 2) return (fp << 2)
| (hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) | (hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0)
| (isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); | (isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0);
} }
private static class PendingEntry { private static class PendingEntry {

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.codecs.lucene90.blocktree;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
@ -237,11 +237,11 @@ final class SegmentTermsEnum extends BaseTermsEnum {
throws IOException { throws IOException {
scratchReader.reset(frameData.bytes, frameData.offset, frameData.length); scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
final long code = scratchReader.readVLong(); final long code = scratchReader.readVLong();
final long fpSeek = code >>> BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS; final long fpSeek = code >>> Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
final SegmentTermsEnumFrame f = getFrame(1 + currentFrame.ord); final SegmentTermsEnumFrame f = getFrame(1 + currentFrame.ord);
f.hasTerms = (code & BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0; f.hasTerms = (code & Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0;
f.hasTermsOrig = f.hasTerms; f.hasTermsOrig = f.hasTerms;
f.isFloor = (code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0; f.isFloor = (code & Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0;
if (f.isFloor) { if (f.isFloor) {
f.setFloorData(scratchReader, frameData); f.setFloorData(scratchReader, frameData);
} }
@ -394,8 +394,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
+ (char) arc.label() + (char) arc.label()
+ " targetLabel=" + " targetLabel="
+ (char) (target.bytes[target.offset + targetUpto] & 0xFF); + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) {
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
} }
if (arc.isFinal()) { if (arc.isFinal()) {
lastFrame = stack[1 + lastFrame.ord]; lastFrame = stack[1 + lastFrame.ord];
@ -491,7 +491,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// term.length = 0; // term.length = 0;
targetUpto = 0; targetUpto = 0;
currentFrame = currentFrame =
pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); pushFrame(
arc, Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
} }
// if (DEBUG) { // if (DEBUG) {
@ -553,8 +554,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
term.setByteAt(targetUpto, (byte) targetLabel); term.setByteAt(targetUpto, (byte) targetLabel);
// Aggregate output as we go: // Aggregate output as we go:
assert arc.output() != null; assert arc.output() != null;
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) {
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
} }
// if (DEBUG) { // if (DEBUG) {
@ -568,7 +569,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
currentFrame = currentFrame =
pushFrame( pushFrame(
arc, arc,
BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()),
targetUpto); targetUpto);
// if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" +
// currentFrame.hasTerms); // currentFrame.hasTerms);
@ -684,8 +685,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// seek; but, often the FST doesn't have any // seek; but, often the FST doesn't have any
// shared bytes (but this could change if we // shared bytes (but this could change if we
// reverse vLong byte order) // reverse vLong byte order)
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) {
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
} }
if (arc.isFinal()) { if (arc.isFinal()) {
lastFrame = stack[1 + lastFrame.ord]; lastFrame = stack[1 + lastFrame.ord];
@ -775,7 +776,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// term.length = 0; // term.length = 0;
targetUpto = 0; targetUpto = 0;
currentFrame = currentFrame =
pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); pushFrame(
arc, Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
} }
// if (DEBUG) { // if (DEBUG) {
@ -837,8 +839,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
arc = nextArc; arc = nextArc;
// Aggregate output as we go: // Aggregate output as we go:
assert arc.output() != null; assert arc.output() != null;
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) {
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
} }
// if (DEBUG) { // if (DEBUG) {
@ -852,7 +854,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
currentFrame = currentFrame =
pushFrame( pushFrame(
arc, arc,
BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()),
targetUpto); targetUpto);
// if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" +
// currentFrame.hasTerms); // currentFrame.hasTerms);
@ -919,9 +921,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
+ " isFloor=" + " isFloor="
+ f.isFloor + f.isFloor
+ " code=" + " code="
+ ((f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + ((f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS)
+ (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0)
+ (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0))
+ " isLastInFloor=" + " isLastInFloor="
+ f.isLastInFloor + f.isLastInFloor
+ " mdUpto=" + " mdUpto="
@ -949,9 +951,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
+ " isFloor=" + " isFloor="
+ f.isFloor + f.isFloor
+ " code=" + " code="
+ ((f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + ((f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS)
+ (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0)
+ (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0))
+ " lastSubFP=" + " lastSubFP="
+ f.lastSubFP + f.lastSubFP
+ " isLastInFloor=" + " isLastInFloor="
@ -980,9 +982,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
new ByteArrayDataInput(output.bytes, output.offset, output.length); new ByteArrayDataInput(output.bytes, output.offset, output.length);
final long codeOrig = reader.readVLong(); final long codeOrig = reader.readVLong();
final long code = final long code =
(f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) (f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS)
| (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) | (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0)
| (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); | (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0);
if (codeOrig != code) { if (codeOrig != code) {
out.println( out.println(
" broken seek state: output code=" " broken seek state: output code="

View File

@ -0,0 +1,777 @@
/*
* 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.lucene90.blocktree;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.fst.FST;
final class SegmentTermsEnumFrame {
// Our index in stack[]:
final int ord;
boolean hasTerms;
boolean hasTermsOrig;
boolean isFloor;
FST.Arc<BytesRef> arc;
// static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
// File pointer where this block was loaded from
long fp;
long fpOrig;
long fpEnd;
long totalSuffixBytes; // for stats
byte[] suffixBytes = new byte[128];
final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
byte[] suffixLengthBytes;
final ByteArrayDataInput suffixLengthsReader;
byte[] statBytes = new byte[64];
int statsSingletonRunLength = 0;
final ByteArrayDataInput statsReader = new ByteArrayDataInput();
byte[] floorData = new byte[32];
final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
// Length of prefix shared by all terms in this block
int prefix;
// Number of entries (term or sub-block) in this block
int entCount;
// Which term we will next read, or -1 if the block
// isn't loaded yet
int nextEnt;
// True if this block is either not a floor block,
// or, it's the last sub-block of a floor block
boolean isLastInFloor;
// True if all entries are terms
boolean isLeafBlock;
long lastSubFP;
int nextFloorLabel;
int numFollowFloorBlocks;
// Next term to decode metaData; we decode metaData
// lazily so that scanning to find the matching term is
// fast and only if you find a match and app wants the
// stats or docs/positions enums, will we decode the
// metaData
int metaDataUpto;
final BlockTermState state;
// metadata buffer
byte[] bytes = new byte[32];
final ByteArrayDataInput bytesReader = new ByteArrayDataInput();
private final SegmentTermsEnum ste;
private final int version;
public SegmentTermsEnumFrame(SegmentTermsEnum ste, int ord) throws IOException {
this.ste = ste;
this.ord = ord;
this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1;
this.version = ste.fr.parent.version;
suffixLengthBytes = new byte[32];
suffixLengthsReader = new ByteArrayDataInput();
}
public void setFloorData(ByteArrayDataInput in, BytesRef source) {
final int numBytes = source.length - (in.getPosition() - source.offset);
if (numBytes > floorData.length) {
floorData = new byte[ArrayUtil.oversize(numBytes, 1)];
}
System.arraycopy(source.bytes, source.offset + in.getPosition(), floorData, 0, numBytes);
floorDataReader.reset(floorData, 0, numBytes);
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
// if (DEBUG) {
// System.out.println(" setFloorData fpOrig=" + fpOrig + " bytes=" + new
// BytesRef(source.bytes, source.offset + in.getPosition(), numBytes) + " numFollowFloorBlocks="
// + numFollowFloorBlocks + " nextFloorLabel=" + toHex(nextFloorLabel));
// }
}
public int getTermBlockOrd() {
return isLeafBlock ? nextEnt : state.termBlockOrd;
}
void loadNextFloorBlock() throws IOException {
// if (DEBUG) {
// System.out.println(" loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
// }
assert arc == null || isFloor : "arc=" + arc + " isFloor=" + isFloor;
fp = fpEnd;
nextEnt = -1;
loadBlock();
}
/* Does initial decode of next block of terms; this
doesn't actually decode the docFreq, totalTermFreq,
postings details (frq/prx offset, etc.) metadata;
it just loads them as byte[] blobs which are then
decoded on-demand if the metadata is ever requested
for any term in this block. This enables terms-only
intensive consumes (eg certain MTQs, respelling) to
not pay the price of decoding metadata they won't
use. */
void loadBlock() throws IOException {
// Clone the IndexInput lazily, so that consumers
// that just pull a TermsEnum to
// seekExact(TermState) don't pay this cost:
ste.initIndexInput();
if (nextEnt != -1) {
// Already loaded
return;
}
// System.out.println("blc=" + blockLoadCount);
ste.in.seek(fp);
int code = ste.in.readVInt();
entCount = code >>> 1;
assert entCount > 0;
isLastInFloor = (code & 1) != 0;
assert arc == null || (isLastInFloor || isFloor)
: "fp=" + fp + " arc=" + arc + " isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
// TODO: if suffixes were stored in random-access
// array structure, then we could do binary search
// instead of linear scan to find target term; eg
// we could have simple array of offsets
final long startSuffixFP = ste.in.getFilePointer();
// term suffixes:
final long codeL = ste.in.readVLong();
isLeafBlock = (codeL & 0x04) != 0;
final int numSuffixBytes = (int) (codeL >>> 3);
if (suffixBytes.length < numSuffixBytes) {
suffixBytes = new byte[ArrayUtil.oversize(numSuffixBytes, 1)];
}
try {
compressionAlg = CompressionAlgorithm.byCode((int) codeL & 0x03);
} catch (IllegalArgumentException e) {
throw new CorruptIndexException(e.getMessage(), ste.in, e);
}
compressionAlg.read(ste.in, suffixBytes, numSuffixBytes);
suffixesReader.reset(suffixBytes, 0, numSuffixBytes);
int numSuffixLengthBytes = ste.in.readVInt();
final boolean allEqual = (numSuffixLengthBytes & 0x01) != 0;
numSuffixLengthBytes >>>= 1;
if (suffixLengthBytes.length < numSuffixLengthBytes) {
suffixLengthBytes = new byte[ArrayUtil.oversize(numSuffixLengthBytes, 1)];
}
if (allEqual) {
Arrays.fill(suffixLengthBytes, 0, numSuffixLengthBytes, ste.in.readByte());
} else {
ste.in.readBytes(suffixLengthBytes, 0, numSuffixLengthBytes);
}
suffixLengthsReader.reset(suffixLengthBytes, 0, numSuffixLengthBytes);
totalSuffixBytes = ste.in.getFilePointer() - startSuffixFP;
/*if (DEBUG) {
if (arc == null) {
System.out.println(" loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
} else {
System.out.println(" loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
}
}*/
// stats
int numBytes = ste.in.readVInt();
if (statBytes.length < numBytes) {
statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ste.in.readBytes(statBytes, 0, numBytes);
statsReader.reset(statBytes, 0, numBytes);
statsSingletonRunLength = 0;
metaDataUpto = 0;
state.termBlockOrd = 0;
nextEnt = 0;
lastSubFP = -1;
// TODO: we could skip this if !hasTerms; but
// that's rare so won't help much
// metadata
numBytes = ste.in.readVInt();
if (bytes.length < numBytes) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ste.in.readBytes(bytes, 0, numBytes);
bytesReader.reset(bytes, 0, numBytes);
// Sub-blocks of a single floor block are always
// written one after another -- tail recurse:
fpEnd = ste.in.getFilePointer();
// if (DEBUG) {
// System.out.println(" fpEnd=" + fpEnd);
// }
}
void rewind() {
// Force reload:
fp = fpOrig;
nextEnt = -1;
hasTerms = hasTermsOrig;
if (isFloor) {
floorDataReader.rewind();
numFollowFloorBlocks = floorDataReader.readVInt();
assert numFollowFloorBlocks > 0;
nextFloorLabel = floorDataReader.readByte() & 0xff;
}
/*
//System.out.println("rewind");
// Keeps the block loaded, but rewinds its state:
if (nextEnt > 0 || fp != fpOrig) {
if (DEBUG) {
System.out.println(" rewind frame ord=" + ord + " fpOrig=" + fpOrig + " fp=" + fp + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " nextEnt=" + nextEnt + " prefixLen=" + prefix);
}
if (fp != fpOrig) {
fp = fpOrig;
nextEnt = -1;
} else {
nextEnt = 0;
}
hasTerms = hasTermsOrig;
if (isFloor) {
floorDataReader.rewind();
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
}
assert suffixBytes != null;
suffixesReader.rewind();
assert statBytes != null;
statsReader.rewind();
metaDataUpto = 0;
state.termBlockOrd = 0;
// TODO: skip this if !hasTerms? Then postings
// impl wouldn't have to write useless 0 byte
postingsReader.resetTermsBlock(fieldInfo, state);
lastSubFP = -1;
} else if (DEBUG) {
System.out.println(" skip rewind fp=" + fp + " fpOrig=" + fpOrig + " nextEnt=" + nextEnt + " ord=" + ord);
}
*/
}
// Decodes next entry; returns true if it's a sub-block
public boolean next() throws IOException {
if (isLeafBlock) {
nextLeaf();
return false;
} else {
return nextNonLeaf();
}
}
public void nextLeaf() {
// if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + "
// entCount=" + entCount);
assert nextEnt != -1 && nextEnt < entCount
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
suffix = suffixLengthsReader.readVInt();
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
ste.termExists = true;
}
public boolean nextNonLeaf() throws IOException {
// if (DEBUG) System.out.println(" stef.next ord=" + ord + " nextEnt=" + nextEnt + " entCount="
// + entCount + " fp=" + suffixesReader.getPosition());
while (true) {
if (nextEnt == entCount) {
assert arc == null || (isFloor && isLastInFloor == false)
: "isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
loadNextFloorBlock();
if (isLeafBlock) {
nextLeaf();
return false;
} else {
continue;
}
}
assert nextEnt != -1 && nextEnt < entCount
: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffix = code >>> 1;
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
if ((code & 1) == 0) {
// A normal term
ste.termExists = true;
subCode = 0;
state.termBlockOrd++;
return false;
} else {
// A sub-block; make sub-FP absolute:
ste.termExists = false;
subCode = suffixLengthsReader.readVLong();
lastSubFP = fp - subCode;
// if (DEBUG) {
// System.out.println(" lastSubFP=" + lastSubFP);
// }
return true;
}
}
}
// TODO: make this array'd so we can do bin search?
// likely not worth it? need to measure how many
// floor blocks we "typically" get
public void scanToFloorFrame(BytesRef target) {
if (!isFloor || target.length <= prefix) {
// if (DEBUG) {
// System.out.println(" scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" +
// target.length + " vs prefix=" + prefix);
// }
return;
}
final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
// if (DEBUG) {
// System.out.println(" scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" +
// toHex(targetLabel) + " vs nextFloorLabel=" + toHex(nextFloorLabel) + " numFollowFloorBlocks="
// + numFollowFloorBlocks);
// }
if (targetLabel < nextFloorLabel) {
// if (DEBUG) {
// System.out.println(" already on correct block");
// }
return;
}
assert numFollowFloorBlocks != 0;
long newFP = fpOrig;
while (true) {
final long code = floorDataReader.readVLong();
newFP = fpOrig + (code >>> 1);
hasTerms = (code & 1) != 0;
// if (DEBUG) {
// System.out.println(" label=" + toHex(nextFloorLabel) + " fp=" + newFP + "
// hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
// }
isLastInFloor = numFollowFloorBlocks == 1;
numFollowFloorBlocks--;
if (isLastInFloor) {
nextFloorLabel = 256;
// if (DEBUG) {
// System.out.println(" stop! last block nextFloorLabel=" +
// toHex(nextFloorLabel));
// }
break;
} else {
nextFloorLabel = floorDataReader.readByte() & 0xff;
if (targetLabel < nextFloorLabel) {
// if (DEBUG) {
// System.out.println(" stop! nextFloorLabel=" + toHex(nextFloorLabel));
// }
break;
}
}
}
if (newFP != fp) {
// Force re-load of the block:
// if (DEBUG) {
// System.out.println(" force switch to fp=" + newFP + " oldFP=" + fp);
// }
nextEnt = -1;
fp = newFP;
} else {
// if (DEBUG) {
// System.out.println(" stay on same fp=" + newFP);
// }
}
}
public void decodeMetaData() throws IOException {
// if (DEBUG) System.out.println("\nBTTR.decodeMetadata seg=" + segment + " mdUpto=" +
// metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd);
// lazily catch up on metadata decode:
final int limit = getTermBlockOrd();
boolean absolute = metaDataUpto == 0;
assert limit > 0;
// TODO: better API would be "jump straight to term=N"???
while (metaDataUpto < limit) {
// TODO: we could make "tiers" of metadata, ie,
// decode docFreq/totalTF but don't decode postings
// metadata; this way caller could get
// docFreq/totalTF w/o paying decode cost for
// postings
// TODO: if docFreq were bulk decoded we could
// just skipN here:
if (statsSingletonRunLength > 0) {
state.docFreq = 1;
state.totalTermFreq = 1;
statsSingletonRunLength--;
} else {
int token = statsReader.readVInt();
if ((token & 1) == 1) {
state.docFreq = 1;
state.totalTermFreq = 1;
statsSingletonRunLength = token >>> 1;
} else {
state.docFreq = token >>> 1;
if (ste.fr.fieldInfo.getIndexOptions() == IndexOptions.DOCS) {
state.totalTermFreq = state.docFreq;
} else {
state.totalTermFreq = state.docFreq + statsReader.readVLong();
}
}
}
// metadata
ste.fr.parent.postingsReader.decodeTerm(bytesReader, ste.fr.fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;
}
state.termBlockOrd = metaDataUpto;
}
// Used only by assert
private boolean prefixMatches(BytesRef target) {
for (int bytePos = 0; bytePos < prefix; bytePos++) {
if (target.bytes[target.offset + bytePos] != ste.term.byteAt(bytePos)) {
return false;
}
}
return true;
}
// Scans to sub-block that has this target fp; only
// called by next(); NOTE: does not set
// startBytePos/suffix as a side effect
public void scanToSubBlock(long subFP) {
assert !isLeafBlock;
// if (DEBUG) System.out.println(" scanToSubBlock fp=" + fp + " subFP=" + subFP + " entCount="
// + entCount + " lastSubFP=" + lastSubFP);
// assert nextEnt == 0;
if (lastSubFP == subFP) {
// if (DEBUG) System.out.println(" already positioned");
return;
}
assert subFP < fp : "fp=" + fp + " subFP=" + subFP;
final long targetSubCode = fp - subFP;
// if (DEBUG) System.out.println(" targetSubCode=" + targetSubCode);
while (true) {
assert nextEnt < entCount;
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffixesReader.skipBytes(code >>> 1);
if ((code & 1) != 0) {
final long subCode = suffixLengthsReader.readVLong();
if (targetSubCode == subCode) {
// if (DEBUG) System.out.println(" match!");
lastSubFP = subFP;
return;
}
} else {
state.termBlockOrd++;
}
}
}
// NOTE: sets startBytePos/suffix as a side effect
public SeekStatus scanToTerm(BytesRef target, boolean exactOnly) throws IOException {
return isLeafBlock ? scanToTermLeaf(target, exactOnly) : scanToTermNonLeaf(target, exactOnly);
}
private int startBytePos;
private int suffix;
private long subCode;
CompressionAlgorithm compressionAlg = CompressionAlgorithm.NO_COMPRESSION;
// for debugging
/*
@SuppressWarnings("unused")
static String brToString(BytesRef b) {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
// If BytesRef isn't actually UTF8, or it's eg a
// prefix of UTF8 that ends mid-unicode-char, we
// fallback to hex:
return b.toString();
}
}
*/
// Target's prefix matches this block's prefix; we
// scan the entries check if the suffix matches.
public SeekStatus scanToTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
// if (DEBUG) System.out.println(" scanToTermLeaf: block fp=" + fp + " prefix=" + prefix + "
// nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" +
// brToString(term));
assert nextEnt != -1;
ste.termExists = true;
subCode = 0;
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
}
return SeekStatus.END;
}
assert prefixMatches(target);
// TODO: binary search when all terms have the same length, which is common for ID fields,
// which are also the most sensitive to lookup performance?
// Loop over each entry (term or sub-block) in this block:
do {
nextEnt++;
suffix = suffixLengthsReader.readVInt();
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
// suffixBytesRef.bytes = suffixBytes;
// suffixBytesRef.offset = suffixesReader.getPosition();
// suffixBytesRef.length = suffix;
// System.out.println(" cycle: term " + (nextEnt-1) + " (of " + entCount + ") suffix="
// + brToString(suffixBytesRef));
// }
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
// Loop over bytes in the suffix, comparing to the target
final int cmp =
Arrays.compareUnsigned(
suffixBytes,
startBytePos,
startBytePos + suffix,
target.bytes,
target.offset + prefix,
target.offset + target.length);
if (cmp < 0) {
// Current entry is still before the target;
// keep scanning
} else if (cmp > 0) {
// Done! Current entry is after target --
// return NOT_FOUND:
fillTerm();
// if (DEBUG) System.out.println(" not found");
return SeekStatus.NOT_FOUND;
} else {
// Exact match!
// This cannot be a sub-block because we
// would have followed the index to this
// sub-block from the start:
assert ste.termExists;
fillTerm();
// if (DEBUG) System.out.println(" found!");
return SeekStatus.FOUND;
}
} while (nextEnt < entCount);
// It is possible (and OK) that terms index pointed us
// at this block, but, we scanned the entire block and
// did not find the term to position to. This happens
// when the target is after the last term in the block
// (but, before the next term in the index). EG
// target could be foozzz, and terms index pointed us
// to the foo* block, but the last term in this block
// was fooz (and, eg, first term in the next block will
// bee fop).
// if (DEBUG) System.out.println(" block end");
if (exactOnly) {
fillTerm();
}
// TODO: not consistent that in the
// not-exact case we don't next() into the next
// frame here
return SeekStatus.END;
}
// Target's prefix matches this block's prefix; we
// scan the entries check if the suffix matches.
public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
// if (DEBUG) System.out.println(" scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix +
// " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" +
// brToString(target));
assert nextEnt != -1;
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
ste.termExists = subCode == 0;
}
return SeekStatus.END;
}
assert prefixMatches(target);
// Loop over each entry (term or sub-block) in this block:
while (nextEnt < entCount) {
nextEnt++;
final int code = suffixLengthsReader.readVInt();
suffix = code >>> 1;
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
// suffixBytesRef.bytes = suffixBytes;
// suffixBytesRef.offset = suffixesReader.getPosition();
// suffixBytesRef.length = suffix;
// System.out.println(" cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " +
// (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
// }
final int termLen = prefix + suffix;
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
ste.termExists = (code & 1) == 0;
if (ste.termExists) {
state.termBlockOrd++;
subCode = 0;
} else {
subCode = suffixLengthsReader.readVLong();
lastSubFP = fp - subCode;
}
final int cmp =
Arrays.compareUnsigned(
suffixBytes,
startBytePos,
startBytePos + suffix,
target.bytes,
target.offset + prefix,
target.offset + target.length);
if (cmp < 0) {
// Current entry is still before the target;
// keep scanning
} else if (cmp > 0) {
// Done! Current entry is after target --
// return NOT_FOUND:
fillTerm();
// if (DEBUG) System.out.println(" maybe done exactOnly=" + exactOnly + "
// ste.termExists=" + ste.termExists);
if (!exactOnly && !ste.termExists) {
// System.out.println(" now pushFrame");
// TODO this
// We are on a sub-block, and caller wants
// 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, termLen);
ste.currentFrame.loadBlock();
while (ste.currentFrame.next()) {
ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
ste.currentFrame.loadBlock();
}
}
// if (DEBUG) System.out.println(" not found");
return SeekStatus.NOT_FOUND;
} else {
// Exact match!
// This cannot be a sub-block because we
// would have followed the index to this
// sub-block from the start:
assert ste.termExists;
fillTerm();
// if (DEBUG) System.out.println(" found!");
return SeekStatus.FOUND;
}
}
// It is possible (and OK) that terms index pointed us
// at this block, but, we scanned the entire block and
// did not find the term to position to. This happens
// when the target is after the last term in the block
// (but, before the next term in the index). EG
// target could be foozzz, and terms index pointed us
// to the foo* block, but the last term in this block
// was fooz (and, eg, first term in the next block will
// bee fop).
// if (DEBUG) System.out.println(" block end");
if (exactOnly) {
fillTerm();
}
// TODO: not consistent that in the
// not-exact case we don't next() into the next
// frame here
return SeekStatus.END;
}
private void fillTerm() {
final int termLength = prefix + suffix;
ste.term.setLength(termLength);
ste.term.grow(termLength);
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix);
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.codecs.blocktree; package org.apache.lucene.codecs.lucene90.blocktree;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.PrintStream; import java.io.PrintStream;

View File

@ -16,7 +16,14 @@
*/ */
/** /**
* Components from the Lucene 8.4 index format. See {@link org.apache.lucene.codecs.lucene90} for an * BlockTree terms dictionary.
* overview of the current index format. *
* <p>This terms dictionary organizes all terms into blocks according to shared prefix, such that
* each block has enough terms, and then stores the prefix trie in memory as an FST as the index
* structure. It allows you to plug in your own {@link org.apache.lucene.codecs.PostingsWriterBase}
* to implement the postings.
*
* <p>See {@link org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter} for the
* file format.
*/ */
package org.apache.lucene.codecs.lucene84; package org.apache.lucene.codecs.lucene90.blocktree;

View File

@ -0,0 +1,522 @@
#! /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 fractions 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.lucene84;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
// Inspired from https://fulmicoton.com/posts/bitpacking/
// Encodes multiple integers in a long to get SIMD-like speedups.
// If bitsPerValue <= 8 then we pack 8 ints per long
// else if bitsPerValue <= 16 we pack 4 ints per long
// else we pack 2 ints per long
final class ForUtil {
static final int BLOCK_SIZE = 128;
private static final int BLOCK_SIZE_LOG2 = 7;
private static long expandMask32(long mask32) {
return mask32 | (mask32 << 32);
}
private static long expandMask16(long mask16) {
return expandMask32(mask16 | (mask16 << 16));
}
private static long expandMask8(long mask8) {
return expandMask16(mask8 | (mask8 << 8));
}
private static long mask32(int bitsPerValue) {
return expandMask32((1L << bitsPerValue) - 1);
}
private static long mask16(int bitsPerValue) {
return expandMask16((1L << bitsPerValue) - 1);
}
private static long mask8(int bitsPerValue) {
return expandMask8((1L << bitsPerValue) - 1);
}
private static void expand8(long[] arr) {
for (int i = 0; i < 16; ++i) {
long l = arr[i];
arr[i] = (l >>> 56) & 0xFFL;
arr[16+i] = (l >>> 48) & 0xFFL;
arr[32+i] = (l >>> 40) & 0xFFL;
arr[48+i] = (l >>> 32) & 0xFFL;
arr[64+i] = (l >>> 24) & 0xFFL;
arr[80+i] = (l >>> 16) & 0xFFL;
arr[96+i] = (l >>> 8) & 0xFFL;
arr[112+i] = l & 0xFFL;
}
}
private static void expand8To32(long[] arr) {
for (int i = 0; i < 16; ++i) {
long l = arr[i];
arr[i] = (l >>> 24) & 0x000000FF000000FFL;
arr[16+i] = (l >>> 16) & 0x000000FF000000FFL;
arr[32+i] = (l >>> 8) & 0x000000FF000000FFL;
arr[48+i] = l & 0x000000FF000000FFL;
}
}
private static void collapse8(long[] arr) {
for (int i = 0; i < 16; ++i) {
arr[i] = (arr[i] << 56) | (arr[16+i] << 48) | (arr[32+i] << 40) | (arr[48+i] << 32) | (arr[64+i] << 24) | (arr[80+i] << 16) | (arr[96+i] << 8) | arr[112+i];
}
}
private static void expand16(long[] arr) {
for (int i = 0; i < 32; ++i) {
long l = arr[i];
arr[i] = (l >>> 48) & 0xFFFFL;
arr[32+i] = (l >>> 32) & 0xFFFFL;
arr[64+i] = (l >>> 16) & 0xFFFFL;
arr[96+i] = l & 0xFFFFL;
}
}
private static void expand16To32(long[] arr) {
for (int i = 0; i < 32; ++i) {
long l = arr[i];
arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL;
arr[32+i] = l & 0x0000FFFF0000FFFFL;
}
}
private static void collapse16(long[] arr) {
for (int i = 0; i < 32; ++i) {
arr[i] = (arr[i] << 48) | (arr[32+i] << 32) | (arr[64+i] << 16) | arr[96+i];
}
}
private static void expand32(long[] arr) {
for (int i = 0; i < 64; ++i) {
long l = arr[i];
arr[i] = l >>> 32;
arr[64 + i] = l & 0xFFFFFFFFL;
}
}
private static void collapse32(long[] arr) {
for (int i = 0; i < 64; ++i) {
arr[i] = (arr[i] << 32) | arr[64+i];
}
}
private static void prefixSum8(long[] arr, long base) {
expand8To32(arr);
prefixSum32(arr, base);
}
private static void prefixSum16(long[] arr, long base) {
// We need to move to the next primitive size to avoid overflows
expand16To32(arr);
prefixSum32(arr, base);
}
private static void prefixSum32(long[] arr, long base) {
arr[0] += base << 32;
innerPrefixSum32(arr);
expand32(arr);
final long l = arr[BLOCK_SIZE/2-1];
for (int i = BLOCK_SIZE/2; i < BLOCK_SIZE; ++i) {
arr[i] += l;
}
}
// For some reason unrolling seems to help
private static void innerPrefixSum32(long[] arr) {
arr[1] += arr[0];
arr[2] += arr[1];
arr[3] += arr[2];
arr[4] += arr[3];
arr[5] += arr[4];
arr[6] += arr[5];
arr[7] += arr[6];
arr[8] += arr[7];
arr[9] += arr[8];
arr[10] += arr[9];
arr[11] += arr[10];
arr[12] += arr[11];
arr[13] += arr[12];
arr[14] += arr[13];
arr[15] += arr[14];
arr[16] += arr[15];
arr[17] += arr[16];
arr[18] += arr[17];
arr[19] += arr[18];
arr[20] += arr[19];
arr[21] += arr[20];
arr[22] += arr[21];
arr[23] += arr[22];
arr[24] += arr[23];
arr[25] += arr[24];
arr[26] += arr[25];
arr[27] += arr[26];
arr[28] += arr[27];
arr[29] += arr[28];
arr[30] += arr[29];
arr[31] += arr[30];
arr[32] += arr[31];
arr[33] += arr[32];
arr[34] += arr[33];
arr[35] += arr[34];
arr[36] += arr[35];
arr[37] += arr[36];
arr[38] += arr[37];
arr[39] += arr[38];
arr[40] += arr[39];
arr[41] += arr[40];
arr[42] += arr[41];
arr[43] += arr[42];
arr[44] += arr[43];
arr[45] += arr[44];
arr[46] += arr[45];
arr[47] += arr[46];
arr[48] += arr[47];
arr[49] += arr[48];
arr[50] += arr[49];
arr[51] += arr[50];
arr[52] += arr[51];
arr[53] += arr[52];
arr[54] += arr[53];
arr[55] += arr[54];
arr[56] += arr[55];
arr[57] += arr[56];
arr[58] += arr[57];
arr[59] += arr[58];
arr[60] += arr[59];
arr[61] += arr[60];
arr[62] += arr[61];
arr[63] += arr[62];
}
private final long[] tmp = new long[BLOCK_SIZE/2];
/**
* Encode 128 integers from {@code longs} into {@code out}.
*/
void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException {
final int nextPrimitive;
final int numLongs;
if (bitsPerValue <= 8) {
nextPrimitive = 8;
numLongs = BLOCK_SIZE / 8;
collapse8(longs);
} else if (bitsPerValue <= 16) {
nextPrimitive = 16;
numLongs = BLOCK_SIZE / 4;
collapse16(longs);
} else {
nextPrimitive = 32;
numLongs = BLOCK_SIZE / 2;
collapse32(longs);
}
final int numLongsPerShift = bitsPerValue * 2;
int idx = 0;
int shift = nextPrimitive - bitsPerValue;
for (int i = 0; i < numLongsPerShift; ++i) {
tmp[i] = longs[idx++] << shift;
}
for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) {
for (int i = 0; i < numLongsPerShift; ++i) {
tmp[i] |= longs[idx++] << shift;
}
}
final int remainingBitsPerLong = shift + bitsPerValue;
final long maskRemainingBitsPerLong;
if (nextPrimitive == 8) {
maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong];
} else if (nextPrimitive == 16) {
maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong];
} else {
maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong];
}
int tmpIdx = 0;
int remainingBitsPerValue = bitsPerValue;
while (idx < numLongs) {
if (remainingBitsPerValue >= remainingBitsPerLong) {
remainingBitsPerValue -= remainingBitsPerLong;
tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong;
if (remainingBitsPerValue == 0) {
idx++;
remainingBitsPerValue = bitsPerValue;
}
} else {
final long mask1, mask2;
if (nextPrimitive == 8) {
mask1 = MASKS8[remainingBitsPerValue];
mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue];
} else if (nextPrimitive == 16) {
mask1 = MASKS16[remainingBitsPerValue];
mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue];
} else {
mask1 = MASKS32[remainingBitsPerValue];
mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue];
}
tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue);
remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue;
tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2;
}
}
for (int i = 0; i < numLongsPerShift; ++i) {
// Java longs are big endian and we want to read little endian longs, so we need to reverse bytes
long l = Long.reverseBytes(tmp[i]);
out.writeLong(l);
}
}
/**
* Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value.
*/
int numBytes(int bitsPerValue) throws IOException {
return bitsPerValue << (BLOCK_SIZE_LOG2 - 3);
}
private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) throws IOException {
final int numLongs = bitsPerValue << 1;
in.readLELongs(tmp, 0, numLongs);
final long mask = MASKS32[bitsPerValue];
int longsIdx = 0;
int shift = 32 - bitsPerValue;
for (; shift >= 0; shift -= bitsPerValue) {
shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask);
longsIdx += numLongs;
}
final int remainingBitsPerLong = shift + bitsPerValue;
final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong];
int tmpIdx = 0;
int remainingBits = remainingBitsPerLong;
for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) {
int b = bitsPerValue - remainingBits;
long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b;
while (b >= remainingBitsPerLong) {
b -= remainingBitsPerLong;
l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b;
}
if (b > 0) {
l |= (tmp[tmpIdx] >>> (remainingBitsPerLong-b)) & MASKS32[b];
remainingBits = remainingBitsPerLong - b;
} else {
remainingBits = remainingBitsPerLong;
}
longs[longsIdx] = l;
}
}
/**
* The pattern that this shiftLongs method applies is recognized by the C2
* compiler, which generates SIMD instructions for it in order to shift
* multiple longs at once.
*/
private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) {
for (int i = 0; i < count; ++i) {
b[bi+i] = (a[i] >>> shift) & mask;
}
}
"""
def writeRemainderWithSIMDOptimize(bpv, next_primitive, remaining_bits_per_long, o, num_values, f):
iteration = 1
num_longs = bpv * num_values / remaining_bits_per_long
while num_longs % 2 == 0 and num_values % 2 == 0:
num_longs /= 2
num_values /= 2
iteration *= 2
f.write(' shiftLongs(tmp, %d, tmp, 0, 0, MASK%d_%d);\n' % (iteration * num_longs, next_primitive, remaining_bits_per_long))
f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values))
tmp_idx = 0
b = bpv
b -= remaining_bits_per_long
f.write(' long l0 = tmp[tmpIdx+%d] << %d;\n' %(tmp_idx, b))
tmp_idx += 1
while b >= remaining_bits_per_long:
b -= remaining_bits_per_long
f.write(' l0 |= tmp[tmpIdx+%d] << %d;\n' %(tmp_idx, b))
tmp_idx += 1
f.write(' longs[longsIdx+0] = l0;\n')
f.write(' }\n')
def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f):
iteration = 1
num_longs = bpv * num_values / remaining_bits_per_long
while num_longs % 2 == 0 and num_values % 2 == 0:
num_longs /= 2
num_values /= 2
iteration *= 2
f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values))
i = 0
remaining_bits = 0
tmp_idx = 0
for i in range(num_values):
b = bpv
if remaining_bits == 0:
b -= remaining_bits_per_long
f.write(' long l%d = (tmp[tmpIdx+%d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b))
else:
b -= remaining_bits
f.write(' long l%d = (tmp[tmpIdx+%d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b))
tmp_idx += 1
while b >= remaining_bits_per_long:
b -= remaining_bits_per_long
f.write(' l%d |= (tmp[tmpIdx+%d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b))
tmp_idx += 1
if b > 0:
f.write(' l%d |= (tmp[tmpIdx+%d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_long-b, next_primitive, b))
remaining_bits = remaining_bits_per_long-b
f.write(' longs[longsIdx+%d] = l%d;\n' %(i, i))
f.write(' }\n')
def writeDecode(bpv, f):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' private static void decode%d(DataInput in, long[] tmp, long[] longs) throws IOException {\n' %bpv)
num_values_per_long = 64 / next_primitive
if bpv == next_primitive:
f.write(' in.readLELongs(longs, 0, %d);\n' %(bpv*2))
else:
f.write(' in.readLELongs(tmp, 0, %d);\n' %(bpv*2))
shift = next_primitive - bpv
o = 0
while shift >= 0:
f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv))
o += bpv*2
shift -= bpv
if shift + bpv > 0:
if bpv % (next_primitive % bpv) == 0:
writeRemainderWithSIMDOptimize(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f)
else:
writeRemainder(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f)
f.write(' }\n')
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(' 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(' }\n')
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\n')
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('\n')
f.write("""
/**
* Decode 128 integers into {@code longs}.
*/
void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException {
switch (bitsPerValue) {
""")
for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' case %d:\n' %bpv)
f.write(' decode%d(in, tmp, longs);\n' %bpv)
f.write(' expand%d(longs);\n' %next_primitive)
f.write(' break;\n')
f.write(' default:\n')
f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n')
f.write(' expand32(longs);\n')
f.write(' break;\n')
f.write(' }\n')
f.write(' }\n')
f.write("""
/**
* Delta-decode 128 integers into {@code longs}.
*/
void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) throws IOException {
switch (bitsPerValue) {
""")
for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
next_primitive = 32
if bpv <= 8:
next_primitive = 8
elif bpv <= 16:
next_primitive = 16
f.write(' case %d:\n' %bpv)
f.write(' decode%d(in, tmp, longs);\n' %bpv)
f.write(' prefixSum%d(longs, base);\n' %next_primitive)
f.write(' break;\n')
f.write(' default:\n')
f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n')
f.write(' prefixSum32(longs, base);\n')
f.write(' break;\n')
f.write(' }\n')
f.write(' }\n')
f.write('\n')
for i in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1):
writeDecode(i, f)
f.write('}\n')

View File

@ -151,15 +151,15 @@
* field names. These are used to store auxiliary information about the document, such as its * 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 * 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. * returned for each hit when searching. This is keyed by document number.
* <li>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term dictionary}. A * <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term dictionary}. A
* dictionary containing all of the terms used in all of the indexed fields of all of the * 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 * documents. The dictionary also contains the number of documents which contain the term, and
* pointers to the term's frequency and proximity data. * pointers to the term's frequency and proximity data.
* <li>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Frequency data}. For * <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Frequency data}. For
* each term in the dictionary, the numbers of all the documents that contain that term, and * 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 * the frequency of the term in that document, unless frequencies are omitted ({@link
* org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
* <li>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Proximity data}. For * <li>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Proximity data}. For
* each term in the dictionary, the positions that the term occurs in each document. Note that * 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. * this will not exist if all fields in all documents omit position data.
* <li>{@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}. For * <li>{@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}. For
@ -255,27 +255,27 @@
* <td>The stored fields for documents</td> * <td>The stored fields for documents</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Dictionary}</td> * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary}</td>
* <td>.tim</td> * <td>.tim</td>
* <td>The term dictionary, stores term info</td> * <td>The term dictionary, stores term info</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Index}</td> * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index}</td>
* <td>.tip</td> * <td>.tip</td>
* <td>The index into the Term Dictionary</td> * <td>The index into the Term Dictionary</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Frequencies}</td> * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies}</td>
* <td>.doc</td> * <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td> * <td>Contains the list of docs which contain each term along with frequency</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Positions}</td> * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions}</td>
* <td>.pos</td> * <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td> * <td>Stores position information about where a term occurs in the index</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Payloads}</td> * <td>{@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads}</td>
* <td>.pay</td> * <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td> * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr> * </tr>

View File

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

View File

@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and # See the License for the specific language governing permissions and
# limitations under the License. # limitations under the License.
org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat

View File

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

View File

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

View File

@ -0,0 +1,145 @@
/*
* 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.lucene90;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.lucene90.Lucene90ScoreSkipReader.MutableImpactList;
import org.apache.lucene.codecs.lucene90.blocktree.FieldReader;
import org.apache.lucene.codecs.lucene90.blocktree.Stats;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Impact;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.TestUtil;
public class TestLucene90PostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene90PostingsFormat());
@Override
protected Codec getCodec() {
return codec;
}
/** Make sure the final sub-block(s) are not skipped. */
public void testFinalBlock() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random())));
for (int i = 0; i < 25; i++) {
Document doc = new Document();
doc.add(newStringField("field", Character.toString((char) (97 + i)), Field.Store.NO));
doc.add(newStringField("field", "z" + Character.toString((char) (97 + i)), Field.Store.NO));
w.addDocument(doc);
}
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
assertEquals(1, r.leaves().size());
FieldReader field = (FieldReader) r.leaves().get(0).reader().terms("field");
// We should see exactly two blocks: one root block (prefix empty string) and one block for z*
// terms (prefix z):
Stats stats = field.getStats();
assertEquals(0, stats.floorBlockCount);
assertEquals(2, stats.nonFloorBlockCount);
r.close();
w.close();
d.close();
}
private void shouldFail(int minItemsInBlock, int maxItemsInBlock) {
expectThrows(
IllegalArgumentException.class,
() -> {
new Lucene90PostingsFormat(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)));
// omit freqs
doTestImpactSerialization(Collections.singletonList(new Impact(1, 42L)));
// omit freqs with very large norms
doTestImpactSerialization(Collections.singletonList(new Impact(1, -100L)));
// omit norms
doTestImpactSerialization(Collections.singletonList(new Impact(30, 1L)));
// omit norms with large freq
doTestImpactSerialization(Collections.singletonList(new Impact(500, 1L)));
// freqs and norms, basic
doTestImpactSerialization(
Arrays.asList(
new Impact(1, 7L),
new Impact(3, 9L),
new Impact(7, 10L),
new Impact(15, 11L),
new Impact(20, 13L),
new Impact(28, 14L)));
// freqs and norms, high values
doTestImpactSerialization(
Arrays.asList(
new Impact(2, 2L),
new Impact(10, 10L),
new Impact(12, 50L),
new Impact(50, -100L),
new Impact(1000, -80L),
new Impact(1005, -3L)));
}
private void doTestImpactSerialization(List<Impact> impacts) throws IOException {
CompetitiveImpactAccumulator acc = new CompetitiveImpactAccumulator();
for (Impact impact : impacts) {
acc.add(impact.freq, impact.norm);
}
try (Directory dir = newDirectory()) {
try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
Lucene90SkipWriter.writeImpacts(acc, out);
}
try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
byte[] b = new byte[Math.toIntExact(in.length())];
in.readBytes(b, 0, b.length);
List<Impact> impacts2 =
Lucene90ScoreSkipReader.readImpacts(new ByteArrayDataInput(b), new MutableImpactList());
assertEquals(impacts, impacts2);
}
}
}
}

View File

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

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Objects; import java.util.Objects;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.PostingsEnum;
@ -261,10 +261,10 @@ class TermIntervalsSource extends IntervalsSource {
/** /**
* A guess of the average number of simple operations for the initial seek and buffer refill per * 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 * document for the positions of a term. See also {@link
* Lucene84PostingsReader.EverythingEnum#nextPosition()}. * Lucene90PostingsReader.EverythingEnum#nextPosition()}.
* *
* <p>Aside: Instead of being constant this could depend among others on {@link * <p>Aside: Instead of being constant this could depend among others on {@link
* Lucene84PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * Lucene90PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link
* TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * 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 * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block
* size of the device storing the index. * size of the device storing the index.
@ -272,7 +272,7 @@ class TermIntervalsSource extends IntervalsSource {
private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128; private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128;
/** /**
* Number of simple operations in {@link Lucene84PostingsReader.EverythingEnum#nextPosition()} * Number of simple operations in {@link Lucene90PostingsReader.EverythingEnum#nextPosition()}
* when no seek or buffer refill is done. * when no seek or buffer refill is done.
*/ */
private static final int TERM_OPS_PER_POS = 7; private static final int TERM_OPS_PER_POS = 7;

View File

@ -22,7 +22,7 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.search.LiveFieldValues; import org.apache.lucene.search.LiveFieldValues;
@ -64,14 +64,16 @@ public class IDVersionPostingsFormat extends PostingsFormat {
private final int maxTermsInBlock; private final int maxTermsInBlock;
public IDVersionPostingsFormat() { public IDVersionPostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); this(
Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
} }
public IDVersionPostingsFormat(int minTermsInBlock, int maxTermsInBlock) { public IDVersionPostingsFormat(int minTermsInBlock, int maxTermsInBlock) {
super("IDVersion"); super("IDVersion");
this.minTermsInBlock = minTermsInBlock; this.minTermsInBlock = minTermsInBlock;
this.maxTermsInBlock = maxTermsInBlock; this.maxTermsInBlock = maxTermsInBlock;
BlockTreeTermsWriter.validateSettings(minTermsInBlock, maxTermsInBlock); Lucene90BlockTreeTermsWriter.validateSettings(minTermsInBlock, maxTermsInBlock);
} }
@Override @Override

View File

@ -24,7 +24,7 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields; import org.apache.lucene.index.Fields;
@ -82,8 +82,8 @@ import org.apache.lucene.util.fst.Util;
*/ */
/** /**
* This is just like {@link BlockTreeTermsWriter}, except it also stores a version per term, and * This is just like {@link Lucene90BlockTreeTermsWriter}, except it also stores a version per term,
* adds a method to its TermsEnum implementation to seekExact only if the version is &gt;= the * and adds a method to its TermsEnum implementation to seekExact only if the version is &gt;= the
* specified version. The version is added to the terms index to avoid seeking if no term in the * specified version. The version is added to the terms index to avoid seeking if no term in the
* block has a high enough version. The term blocks file is .tiv and the terms index extension is * block has a high enough version. The term blocks file is .tiv and the terms index extension is
* .tipv. * .tipv.
@ -181,7 +181,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
int minItemsInBlock, int minItemsInBlock,
int maxItemsInBlock) int maxItemsInBlock)
throws IOException { throws IOException {
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); Lucene90BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
maxDoc = state.segmentInfo.maxDoc(); maxDoc = state.segmentInfo.maxDoc();
final String termsFileName = final String termsFileName =

View File

@ -20,9 +20,9 @@ import org.apache.lucene.codecs.PostingsFormat;
/** /**
* {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code
* org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat}. This format is only used for * org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat}. This format is only used for
* backward-compatibility of the index format and cannot be used to write data, use {@link * backward-compatibility of the index format and cannot be used to write data, use {@link
* Completion84PostingsFormat} on new indices. * Completion90PostingsFormat} on new indices.
* *
* @lucene.experimental * @lucene.experimental
*/ */

View File

@ -19,8 +19,10 @@ package org.apache.lucene.search.suggest.document;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
/** /**
* {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@link * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code
* org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat} * org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat}. This format is only used for
* backward-compatibility of the index format and cannot be used to write data, use {@link
* Completion90PostingsFormat} on new indices.
* *
* @lucene.experimental * @lucene.experimental
*/ */

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search.suggest.document;
import org.apache.lucene.codecs.PostingsFormat;
/**
* {@link CompletionPostingsFormat} for {@link
* org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat}
*
* @lucene.experimental
*/
public class Completion90PostingsFormat extends CompletionPostingsFormat {
/** Creates a {@link Completion90PostingsFormat} that will load the completion FST on-heap. */
public Completion90PostingsFormat() {
this(FSTLoadMode.ON_HEAP);
}
/**
* Creates a {@link Completion90PostingsFormat} that will use the provided <code>fstLoadMode
* </code> to determine if the completion FST should be loaded on or off heap.
*/
public Completion90PostingsFormat(FSTLoadMode fstLoadMode) {
super("Completion90", fstLoadMode);
}
@Override
protected PostingsFormat delegatePostingsFormat() {
return PostingsFormat.forName("Lucene90");
}
}

View File

@ -32,3 +32,4 @@
org.apache.lucene.search.suggest.document.Completion50PostingsFormat org.apache.lucene.search.suggest.document.Completion50PostingsFormat
org.apache.lucene.search.suggest.document.Completion84PostingsFormat org.apache.lucene.search.suggest.document.Completion84PostingsFormat
org.apache.lucene.search.suggest.document.Completion90PostingsFormat

View File

@ -964,7 +964,7 @@ public class TestSuggestField extends LuceneTestCase {
new Lucene90Codec() { new Lucene90Codec() {
CompletionPostingsFormat.FSTLoadMode fstLoadMode = CompletionPostingsFormat.FSTLoadMode fstLoadMode =
RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
PostingsFormat postingsFormat = new Completion84PostingsFormat(fstLoadMode); PostingsFormat postingsFormat = new Completion90PostingsFormat(fstLoadMode);
@Override @Override
public PostingsFormat getPostingsFormatForField(String field) { public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsFormat and make it ord-able... // any PostingsFormat and make it ord-able...
/** /**
* Customized version of {@link Lucene84PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. * Customized version of {@link Lucene90PostingsFormat} that uses {@link FixedGapTermsIndexWriter}.
*/ */
public final class LuceneFixedGap extends PostingsFormat { public final class LuceneFixedGap extends PostingsFormat {
final int termIndexInterval; final int termIndexInterval;
@ -48,7 +48,7 @@ public final class LuceneFixedGap extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene84PostingsWriter(state); PostingsWriterBase docs = new Lucene90PostingsWriter(state);
// TODO: should we make the terms index more easily // TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which // pluggable? Ie so that this codec would record which
@ -85,7 +85,7 @@ public final class LuceneFixedGap extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene84PostingsReader(state); PostingsReaderBase postings = new Lucene90PostingsReader(state);
TermsIndexReaderBase indexReader; TermsIndexReaderBase indexReader;
boolean success = false; boolean success = false;

View File

@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsFormat and make it ord-able... // any PostingsFormat and make it ord-able...
/** /**
* Customized version of {@link Lucene84PostingsFormat} that uses {@link * Customized version of {@link Lucene90PostingsFormat} that uses {@link
* VariableGapTermsIndexWriter} with a fixed interval, but forcing high docfreq terms to be indexed * VariableGapTermsIndexWriter} with a fixed interval, but forcing high docfreq terms to be indexed
* terms. * terms.
*/ */
@ -52,7 +52,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene84PostingsWriter(state); PostingsWriterBase docs = new Lucene90PostingsWriter(state);
// TODO: should we make the terms index more easily // TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which // pluggable? Ie so that this codec would record which
@ -93,7 +93,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene84PostingsReader(state); PostingsReaderBase postings = new Lucene90PostingsReader(state);
TermsIndexReaderBase indexReader; TermsIndexReaderBase indexReader;
boolean success = false; boolean success = false;

View File

@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState;
// any PostingsFormat and make it ord-able... // any PostingsFormat and make it ord-able...
/** /**
* Customized version of {@link Lucene84PostingsFormat} that uses {@link * Customized version of {@link Lucene90PostingsFormat} that uses {@link
* VariableGapTermsIndexWriter} with a fixed interval. * VariableGapTermsIndexWriter} with a fixed interval.
*/ */
public final class LuceneVarGapFixedInterval extends PostingsFormat { public final class LuceneVarGapFixedInterval extends PostingsFormat {
@ -49,7 +49,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene84PostingsWriter(state); PostingsWriterBase docs = new Lucene90PostingsWriter(state);
// TODO: should we make the terms index more easily // TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which // pluggable? Ie so that this codec would record which
@ -88,7 +88,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene84PostingsReader(state); PostingsReaderBase postings = new Lucene90PostingsReader(state);
TermsIndexReaderBase indexReader; TermsIndexReaderBase indexReader;
boolean success = false; boolean success = false;

View File

@ -33,12 +33,12 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader;
import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader;
import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter;
import org.apache.lucene.codecs.memory.FSTTermsReader; import org.apache.lucene.codecs.memory.FSTTermsReader;
import org.apache.lucene.codecs.memory.FSTTermsWriter; import org.apache.lucene.codecs.memory.FSTTermsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
@ -121,7 +121,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
random.nextInt(); // consume a random for buffersize random.nextInt(); // consume a random for buffersize
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state);
final FieldsConsumer fields; final FieldsConsumer fields;
final int t1 = random.nextInt(4); final int t1 = random.nextInt(4);
@ -150,7 +150,9 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
boolean success = false; boolean success = false;
try { try {
fields = new BlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock); fields =
new Lucene90BlockTreeTermsWriter(
state, postingsWriter, minTermsInBlock, maxTermsInBlock);
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {
@ -287,7 +289,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize);
} }
PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); PostingsReaderBase postingsReader = new Lucene90PostingsReader(state);
final FieldsProducer fields; final FieldsProducer fields;
final int t1 = random.nextInt(4); final int t1 = random.nextInt(4);
@ -309,7 +311,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
boolean success = false; boolean success = false;
try { try {
fields = new BlockTreeTermsReader(postingsReader, state); fields = new Lucene90BlockTreeTermsReader(postingsReader, state);
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {

View File

@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput;
@ -61,7 +61,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat {
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException {
PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(segmentWriteState); PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(segmentWriteState);
boolean success = false; boolean success = false;
try { try {
FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter); FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter);
@ -139,7 +139,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat {
@Override @Override
public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException { public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException {
PostingsReaderBase postingsReader = new Lucene84PostingsReader(segmentReadState); PostingsReaderBase postingsReader = new Lucene90PostingsReader(segmentReadState);
boolean success = false; boolean success = false;
try { try {
FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader); FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader);

View File

@ -54,8 +54,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap; import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90Codec; import org.apache.lucene.codecs.lucene90.Lucene90Codec;
import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.BinaryDocValuesField;
@ -1219,7 +1219,7 @@ public final class TestUtil {
* Lucene. * Lucene.
*/ */
public static PostingsFormat getDefaultPostingsFormat() { public static PostingsFormat getDefaultPostingsFormat() {
return new Lucene84PostingsFormat(); return new Lucene90PostingsFormat();
} }
/** /**
@ -1230,7 +1230,7 @@ public final class TestUtil {
*/ */
public static PostingsFormat getDefaultPostingsFormat( public static PostingsFormat getDefaultPostingsFormat(
int minItemsPerBlock, int maxItemsPerBlock) { int minItemsPerBlock, int maxItemsPerBlock) {
return new Lucene84PostingsFormat(minItemsPerBlock, maxItemsPerBlock); return new Lucene90PostingsFormat(minItemsPerBlock, maxItemsPerBlock);
} }
/** Returns a random postings format that supports term ordinals */ /** Returns a random postings format that supports term ordinals */

View File

@ -20,7 +20,7 @@
as a way to vet that the configuration actually matters. as a way to vet that the configuration actually matters.
--> -->
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct" docValuesFormat="Asserting" /> <fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct" docValuesFormat="Asserting" />
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene84"/> <fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene90"/>
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene80"/> <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene80"/>