mirror of https://github.com/apache/lucene.git
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:
parent
f783848e71
commit
f43fe7642e
|
@ -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;
|
||||||
|
}
|
|
@ -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);
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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;
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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 {
|
||||||
|
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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 {
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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 {
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
|
@ -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();
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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)));
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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 >= the
|
* and adds a method to its TermsEnum implementation to seekExact only if the version is >= 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();
|
||||||
|
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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%).
|
||||||
|
|
|
@ -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 =
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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: <document number, frequency>, <position,
|
||||||
|
* payload length>, <position, offset start, offset length>, and <position,
|
||||||
|
* payload length, offsetstart, offset length>.
|
||||||
|
* <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 --> Header, PackedBlockSize
|
||||||
|
* <li>TermMetadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?,
|
||||||
|
* PayFPDelta?, SkipFPDelta?
|
||||||
|
* <li>Header, --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||||
|
* <li>PackedBlockSize, SingletonDocID --> {@link DataOutput#writeVInt VInt}
|
||||||
|
* <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --> {@link
|
||||||
|
* DataOutput#writeVLong VLong}
|
||||||
|
* <li>Footer --> {@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 <TermPayloads,
|
||||||
|
* TermOffsets?> within the .pay file. Similar to DocFPDelta, it is the difference
|
||||||
|
* between two file positions (or neglected, for fields that omit payloads and offsets).
|
||||||
|
* <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) --> Header, <TermFreqs, SkipData?><sup>TermCount</sup>, Footer
|
||||||
|
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||||
|
* <li>TermFreqs --> <PackedBlock> <sup>PackedDocBlockNum</sup>, VIntBlock?
|
||||||
|
* <li>PackedBlock --> PackedDocDeltaBlock, PackedFreqBlock?
|
||||||
|
* <li>VIntBlock --> <DocDelta[,
|
||||||
|
* Freq?]><sup>DocFreq-PackedBlockSize*PackedDocBlockNum</sup>
|
||||||
|
* <li>SkipData --> <<SkipLevelLength, SkipLevel> <sup>NumSkipLevels-1</sup>,
|
||||||
|
* SkipLevel>, SkipDatum?
|
||||||
|
* <li>SkipLevel --> <SkipDatum> <sup>TrimmedDocFreq/(PackedBlockSize^(Level +
|
||||||
|
* 1))</sup>
|
||||||
|
* <li>SkipDatum --> DocSkip, DocFPSkip, <PosFPSkip, PosBlockOffset, PayLength?,
|
||||||
|
* PayFPSkip?>?, SkipChildLevelPointer?
|
||||||
|
* <li>PackedDocDeltaBlock, PackedFreqBlock --> {@link PackedInts PackedInts}
|
||||||
|
* <li>DocDelta, Freq, DocSkip, DocFPSkip, PosFPSkip, PosBlockOffset, PayByteUpto, PayFPSkip
|
||||||
|
* --> {@link DataOutput#writeVInt VInt}
|
||||||
|
* <li>SkipChildLevelPointer --> {@link DataOutput#writeVLong VLong}
|
||||||
|
* <li>Footer --> {@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) --> Header, <TermPositions> <sup>TermCount</sup>, Footer
|
||||||
|
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||||
|
* <li>TermPositions --> <PackedPosDeltaBlock> <sup>PackedPosBlockNum</sup>,
|
||||||
|
* VIntBlock?
|
||||||
|
* <li>VIntBlock --> <PositionDelta[, PayloadLength?], PayloadData?, OffsetDelta?,
|
||||||
|
* OffsetLength?><sup>PosVIntCount</sup>
|
||||||
|
* <li>PackedPosDeltaBlock --> {@link PackedInts PackedInts}
|
||||||
|
* <li>PositionDelta, OffsetDelta, OffsetLength --> {@link DataOutput#writeVInt VInt}
|
||||||
|
* <li>PayloadData --> {@link DataOutput#writeByte byte}<sup>PayLength</sup>
|
||||||
|
* <li>Footer --> {@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): --> Header, <TermPayloads, TermOffsets?>
|
||||||
|
* <sup>TermCount</sup>, Footer
|
||||||
|
* <li>Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
|
||||||
|
* <li>TermPayloads --> <PackedPayLengthBlock, SumPayLength, PayData>
|
||||||
|
* <sup>PackedPayBlockNum</sup>
|
||||||
|
* <li>TermOffsets --> <PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock>
|
||||||
|
* <sup>PackedPayBlockNum</sup>
|
||||||
|
* <li>PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock -->
|
||||||
|
* {@link PackedInts PackedInts}
|
||||||
|
* <li>SumPayLength --> {@link DataOutput#writeVInt VInt}
|
||||||
|
* <li>PayData --> {@link DataOutput#writeByte byte}<sup>SumPayLength</sup>
|
||||||
|
* <li>Footer --> {@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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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
|
||||||
|
+ ")";
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
||||||
|
+ ")";
|
||||||
|
}
|
||||||
|
}
|
|
@ -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 {
|
|
@ -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="
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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')
|
|
@ -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>
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
|
@ -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;
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 >= the
|
* and adds a method to its TermsEnum implementation to seekExact only if the version is >= 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 =
|
||||||
|
|
|
@ -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
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -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
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -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");
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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 */
|
||||||
|
|
|
@ -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"/>
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue