diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/CompressionAlgorithm.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/CompressionAlgorithm.java new file mode 100644 index 00000000000..ceb1838dd9d --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/CompressionAlgorithm.java @@ -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; +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java similarity index 96% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java index 55d86dfda9a..8c5763fd794 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.backward_codecs.lucene40.blocktree; import java.io.IOException; import java.util.Collection; @@ -56,13 +56,13 @@ public final class FieldReader extends Terms implements Accountable { final BytesRef rootCode; final BytesRef minTerm; final BytesRef maxTerm; - final BlockTreeTermsReader parent; + final Lucene40BlockTreeTermsReader parent; final FST index; // private boolean DEBUG; FieldReader( - BlockTreeTermsReader parent, + Lucene40BlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, BytesRef rootCode, @@ -92,7 +92,7 @@ public final class FieldReader extends Terms implements Accountable { // } rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() - >>> BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS; + >>> Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS; // Initialize FST always off-heap. final IndexInput clone = indexIn.clone(); clone.seek(indexStartFP); diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnum.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnum.java new file mode 100644 index 00000000000..2c381ed1795 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnum.java @@ -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 fstOutputs = ByteSequenceOutputs.getSingleton(); + + IntersectTermsEnumFrame[] stack; + + @SuppressWarnings({"rawtypes", "unchecked"}) + private FST.Arc[] 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 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 getArc(int ord) { + if (ord >= arcs.length) { + @SuppressWarnings({"rawtypes", "unchecked"}) + final FST.Arc[] 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 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 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(); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnumFrame.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnumFrame.java index 63265e403d8..d4c470c40a0 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/IntersectTermsEnumFrame.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.backward_codecs.lucene40.blocktree; import java.io.IOException; import java.util.Arrays; @@ -105,7 +105,7 @@ final class IntersectTermsEnumFrame { this.termState = ite.fr.parent.postingsReader.newTermState(); this.termState.totalTermFreq = -1; this.version = ite.fr.parent.version; - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { suffixLengthBytes = new byte[32]; suffixLengthsReader = new ByteArrayDataInput(); } else { @@ -154,7 +154,7 @@ final class IntersectTermsEnumFrame { // Skip first long -- has redundant fp, hasTerms // flag, isFloor flag final long code = floorDataReader.readVLong(); - if ((code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) { + if ((code & Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) { // Floor frame numFollowFloorBlocks = floorDataReader.readVInt(); nextFloorLabel = floorDataReader.readByte() & 0xff; @@ -184,7 +184,7 @@ final class IntersectTermsEnumFrame { isLastInFloor = (code & 1) != 0; // term suffixes: - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { final long codeL = ite.in.readVLong(); isLeafBlock = (codeL & 0x04) != 0; final int numSuffixBytes = (int) (codeL >>> 3); @@ -315,14 +315,15 @@ final class IntersectTermsEnumFrame { // just skipN here: // stats - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (statsSingletonRunLength > 0) { termState.docFreq = 1; termState.totalTermFreq = 1; statsSingletonRunLength--; } else { 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.totalTermFreq = 1; statsSingletonRunLength = token >>> 1; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java index 79916f8351d..c66daa21a43 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.backward_codecs.lucene40.blocktree; import java.io.IOException; import java.util.ArrayList; @@ -60,11 +60,11 @@ import org.apache.lucene.util.fst.Outputs; *

Use {@link org.apache.lucene.index.CheckIndex} with the -verbose option to see * summary statistics on the blocks in the dictionary. * - *

See {@link BlockTreeTermsWriter}. + *

See {@code BlockTreeTermsWriter}. * * @lucene.experimental */ -public final class BlockTreeTermsReader extends FieldsProducer { +public final class Lucene40BlockTreeTermsReader extends FieldsProducer { static final Outputs FST_OUTPUTS = ByteSequenceOutputs.getSingleton(); @@ -124,7 +124,7 @@ public final class BlockTreeTermsReader extends FieldsProducer { final int version; /** Sole constructor. */ - public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) + public Lucene40BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException { boolean success = false; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnum.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnum.java new file mode 100644 index 00000000000..52c0e339128 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnum.java @@ -0,0 +1,1193 @@ +/* + * 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 java.io.PrintStream; +import org.apache.lucene.codecs.BlockTermState; +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.store.ByteArrayDataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.Util; + +/** Iterates through terms in this field. */ +final class SegmentTermsEnum extends BaseTermsEnum { + + // Lazy init: + IndexInput in; + + private SegmentTermsEnumFrame[] stack; + private final SegmentTermsEnumFrame staticFrame; + SegmentTermsEnumFrame currentFrame; + boolean termExists; + final FieldReader fr; + + private int targetBeforeCurrentLength; + + // static boolean DEBUG = BlockTreeTermsWriter.DEBUG; + + private final ByteArrayDataInput scratchReader = new ByteArrayDataInput(); + + // What prefix of the current term was present in the index; when we only next() through the + // index, this stays at 0. It's only set when + // we seekCeil/Exact: + private int validIndexPrefix; + + // assert only: + private boolean eof; + + final BytesRefBuilder term = new BytesRefBuilder(); + private final FST.BytesReader fstReader; + + @SuppressWarnings({"rawtypes", "unchecked"}) + private FST.Arc[] arcs = new FST.Arc[1]; + + public SegmentTermsEnum(FieldReader fr) throws IOException { + this.fr = fr; + + // if (DEBUG) { + // System.out.println("BTTR.init seg=" + fr.parent.segment); + // } + stack = new SegmentTermsEnumFrame[0]; + + // Used to hold seek by TermState, or cached seek + staticFrame = new SegmentTermsEnumFrame(this, -1); + + if (fr.index == null) { + fstReader = null; + } else { + fstReader = fr.index.getBytesReader(); + } + + // Init w/ root block; don't use index since it may + // not (and need not) have been loaded + for (int arcIdx = 0; arcIdx < arcs.length; arcIdx++) { + arcs[arcIdx] = new FST.Arc<>(); + } + + currentFrame = staticFrame; + final FST.Arc arc; + if (fr.index != null) { + arc = fr.index.getFirstArc(arcs[0]); + // Empty string prefix must have an output in the index! + assert arc.isFinal(); + } else { + arc = null; + } + // currentFrame = pushFrame(arc, rootCode, 0); + // currentFrame.loadBlock(); + validIndexPrefix = 0; + // if (DEBUG) { + // System.out.println("init frame state " + currentFrame.ord); + // printSeekState(); + // } + + // System.out.println(); + // computeBlockStats().print(System.out); + } + + // Not private to avoid synthetic access$NNN methods + void initIndexInput() { + if (this.in == null) { + this.in = fr.parent.termsIn.clone(); + } + } + + /** Runs next() through the entire terms dict, computing aggregate statistics. */ + public Stats computeBlockStats() throws IOException { + + Stats stats = new Stats(fr.parent.segment, fr.fieldInfo.name); + if (fr.index != null) { + stats.indexNumBytes = fr.index.ramBytesUsed(); + } + + currentFrame = staticFrame; + FST.Arc arc; + if (fr.index != null) { + arc = fr.index.getFirstArc(arcs[0]); + // Empty string prefix must have an output in the index! + assert arc.isFinal(); + } else { + arc = null; + } + + // Empty string prefix must have an output in the + // index! + currentFrame = pushFrame(arc, fr.rootCode, 0); + currentFrame.fpOrig = currentFrame.fp; + currentFrame.loadBlock(); + validIndexPrefix = 0; + + stats.startBlock(currentFrame, !currentFrame.isLastInFloor); + + allTerms: + while (true) { + + // Pop finished blocks + while (currentFrame.nextEnt == currentFrame.entCount) { + stats.endBlock(currentFrame); + if (!currentFrame.isLastInFloor) { + // Advance to next floor block + currentFrame.loadNextFloorBlock(); + stats.startBlock(currentFrame, true); + break; + } else { + if (currentFrame.ord == 0) { + break allTerms; + } + final long lastFP = currentFrame.fpOrig; + currentFrame = stack[currentFrame.ord - 1]; + assert lastFP == currentFrame.lastSubFP; + // if (DEBUG) { + // System.out.println(" reset validIndexPrefix=" + validIndexPrefix); + // } + } + } + + while (true) { + if (currentFrame.next()) { + // Push to new block: + currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length()); + currentFrame.fpOrig = currentFrame.fp; + // This is a "next" frame -- even if it's + // floor'd we must pretend it isn't so we don't + // try to scan to the right floor frame: + currentFrame.loadBlock(); + stats.startBlock(currentFrame, !currentFrame.isLastInFloor); + } else { + stats.term(term.get()); + break; + } + } + } + + stats.finish(); + + // Put root frame back: + currentFrame = staticFrame; + if (fr.index != null) { + arc = fr.index.getFirstArc(arcs[0]); + // Empty string prefix must have an output in the index! + assert arc.isFinal(); + } else { + arc = null; + } + currentFrame = pushFrame(arc, fr.rootCode, 0); + currentFrame.rewind(); + currentFrame.loadBlock(); + validIndexPrefix = 0; + term.clear(); + + return stats; + } + + private SegmentTermsEnumFrame getFrame(int ord) throws IOException { + if (ord >= stack.length) { + final SegmentTermsEnumFrame[] next = + new SegmentTermsEnumFrame + [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 SegmentTermsEnumFrame(this, stackOrd); + } + stack = next; + } + assert stack[ord].ord == ord; + return stack[ord]; + } + + private FST.Arc getArc(int ord) { + if (ord >= arcs.length) { + @SuppressWarnings({"rawtypes", "unchecked"}) + final FST.Arc[] 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]; + } + + // Pushes a frame we seek'd to + SegmentTermsEnumFrame pushFrame(FST.Arc arc, BytesRef frameData, int length) + throws IOException { + scratchReader.reset(frameData.bytes, frameData.offset, frameData.length); + final long code = scratchReader.readVLong(); + final long fpSeek = code >>> Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS; + final SegmentTermsEnumFrame f = getFrame(1 + currentFrame.ord); + f.hasTerms = (code & Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0; + f.hasTermsOrig = f.hasTerms; + f.isFloor = (code & Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0; + if (f.isFloor) { + f.setFloorData(scratchReader, frameData); + } + pushFrame(arc, fpSeek, length); + + return f; + } + + // Pushes next'd frame or seek'd frame; we later + // lazy-load the frame only when needed + SegmentTermsEnumFrame pushFrame(FST.Arc arc, long fp, int length) throws IOException { + final SegmentTermsEnumFrame f = getFrame(1 + currentFrame.ord); + f.arc = arc; + if (f.fpOrig == fp && f.nextEnt != -1) { + // if (DEBUG) System.out.println(" push reused frame ord=" + f.ord + " fp=" + f.fp + " + // isFloor?=" + f.isFloor + " hasTerms=" + f.hasTerms + " pref=" + term + " nextEnt=" + + // f.nextEnt + " targetBeforeCurrentLength=" + targetBeforeCurrentLength + " term.length=" + + // term.length + " vs prefix=" + f.prefix); + // if (f.prefix > targetBeforeCurrentLength) { + if (f.ord > targetBeforeCurrentLength) { + f.rewind(); + } else { + // if (DEBUG) { + // System.out.println(" skip rewind!"); + // } + } + assert length == f.prefix; + } else { + f.nextEnt = -1; + f.prefix = length; + f.state.termBlockOrd = 0; + f.fpOrig = f.fp = fp; + f.lastSubFP = -1; + // if (DEBUG) { + // final int sav = term.length; + // term.length = length; + // System.out.println(" push new frame ord=" + f.ord + " fp=" + f.fp + " hasTerms=" + + // f.hasTerms + " isFloor=" + f.isFloor + " pref=" + brToString(term)); + // term.length = sav; + // } + } + + return f; + } + + // asserts only + private boolean clearEOF() { + eof = false; + return true; + } + + // asserts only + private boolean setEOF() { + eof = true; + return true; + } + + /* + // 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(); + } + } + + // for debugging + @SuppressWarnings("unused") + static String brToString(BytesRefBuilder b) { + return brToString(b.get()); + } + */ + + @Override + public boolean seekExact(BytesRef target) throws IOException { + + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (fr.size() > 0 && (target.compareTo(fr.getMin()) < 0 || target.compareTo(fr.getMax()) > 0)) { + return false; + } + + term.grow(1 + target.length); + + assert clearEOF(); + + // if (DEBUG) { + // System.out.println("\nBTTR.seekExact seg=" + fr.parent.segment + " target=" + + // fr.fieldInfo.name + ":" + brToString(target) + " current=" + brToString(term) + " (exists?=" + // + termExists + ") validIndexPrefix=" + validIndexPrefix); + // printSeekState(System.out); + // } + + FST.Arc arc; + int targetUpto; + BytesRef output; + + targetBeforeCurrentLength = currentFrame.ord; + + if (currentFrame != staticFrame) { + + // We are already seek'd; find the common + // prefix of new seek term vs current term and + // re-use the corresponding seek state. For + // example, if app first seeks to foobar, then + // seeks to foobaz, we can re-use the seek state + // for the first 5 bytes. + + // if (DEBUG) { + // System.out.println(" re-use current seek state validIndexPrefix=" + validIndexPrefix); + // } + + arc = arcs[0]; + assert arc.isFinal(); + output = arc.output(); + targetUpto = 0; + + SegmentTermsEnumFrame lastFrame = stack[0]; + assert validIndexPrefix <= term.length(); + + final int targetLimit = Math.min(target.length, validIndexPrefix); + + int cmp = 0; + + // TODO: reverse vLong byte order for better FST + // prefix output sharing + + // First compare up to valid seek frames: + while (targetUpto < targetLimit) { + cmp = (term.byteAt(targetUpto) & 0xFF) - (target.bytes[target.offset + targetUpto] & 0xFF); + // if (DEBUG) { + // System.out.println(" cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + // + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + + // " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")" + " arc.output=" + arc.output + // + " output=" + output); + // } + if (cmp != 0) { + break; + } + arc = arcs[1 + targetUpto]; + assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF) + : "arc.label=" + + (char) arc.label() + + " targetLabel=" + + (char) (target.bytes[target.offset + targetUpto] & 0xFF); + if (arc.output() != Lucene40BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + } + if (arc.isFinal()) { + lastFrame = stack[1 + lastFrame.ord]; + } + targetUpto++; + } + + if (cmp == 0) { + final int targetUptoMid = targetUpto; + + // Second compare the rest of the term, but + // don't save arc/output/frame; we only do this + // to find out if the target term is before, + // equal or after the current term + final int targetLimit2 = Math.min(target.length, term.length()); + while (targetUpto < targetLimit2) { + cmp = + (term.byteAt(targetUpto) & 0xFF) - (target.bytes[target.offset + targetUpto] & 0xFF); + // if (DEBUG) { + // System.out.println(" cycle2 targetUpto=" + targetUpto + " (vs limit=" + + // targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + + // targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"); + // } + if (cmp != 0) { + break; + } + targetUpto++; + } + + if (cmp == 0) { + cmp = term.length() - target.length; + } + targetUpto = targetUptoMid; + } + + if (cmp < 0) { + // Common case: target term is after current + // term, ie, app is seeking multiple terms + // in sorted order + // if (DEBUG) { + // System.out.println(" target is after current (shares prefixLen=" + targetUpto + "); + // frame.ord=" + lastFrame.ord); + // } + currentFrame = lastFrame; + + } else if (cmp > 0) { + // Uncommon case: target term + // is before current term; this means we can + // keep the currentFrame but we must rewind it + // (so we scan from the start) + targetBeforeCurrentLength = lastFrame.ord; + // if (DEBUG) { + // System.out.println(" target is before current (shares prefixLen=" + targetUpto + "); + // rewind frame ord=" + lastFrame.ord); + // } + currentFrame = lastFrame; + currentFrame.rewind(); + } else { + // Target is exactly the same as current term + assert term.length() == target.length; + if (termExists) { + // if (DEBUG) { + // System.out.println(" target is same as current; return true"); + // } + return true; + } else { + // if (DEBUG) { + // System.out.println(" target is same as current but term doesn't exist"); + // } + } + // validIndexPrefix = currentFrame.depth; + // term.length = target.length; + // return termExists; + } + + } else { + + targetBeforeCurrentLength = -1; + arc = fr.index.getFirstArc(arcs[0]); + + // Empty string prefix must have an output (block) in the index! + assert arc.isFinal(); + assert arc.output() != null; + + // if (DEBUG) { + // System.out.println(" no seek state; push root frame"); + // } + + output = arc.output(); + + currentFrame = staticFrame; + + // term.length = 0; + targetUpto = 0; + currentFrame = + pushFrame( + arc, Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); + } + + // if (DEBUG) { + // System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " + // currentFrame.ord=" + currentFrame.ord + " targetBeforeCurrentLength=" + + // targetBeforeCurrentLength); + // } + + // We are done sharing the common prefix with the incoming target and where we are currently + // seek'd; now continue walking the index: + while (targetUpto < target.length) { + + final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF; + + final FST.Arc nextArc = + fr.index.findTargetArc(targetLabel, arc, getArc(1 + targetUpto), fstReader); + + if (nextArc == null) { + + // Index is exhausted + // if (DEBUG) { + // System.out.println(" index: index exhausted label=" + ((char) targetLabel) + " " + + // toHex(targetLabel)); + // } + + validIndexPrefix = currentFrame.prefix; + // validIndexPrefix = targetUpto; + + currentFrame.scanToFloorFrame(target); + + if (!currentFrame.hasTerms) { + termExists = false; + term.setByteAt(targetUpto, (byte) targetLabel); + term.setLength(1 + targetUpto); + // if (DEBUG) { + // System.out.println(" FAST NOT_FOUND term=" + brToString(term)); + // } + return false; + } + + currentFrame.loadBlock(); + + final SeekStatus result = currentFrame.scanToTerm(target, true); + if (result == SeekStatus.FOUND) { + // if (DEBUG) { + // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term); + // } + return true; + } else { + // if (DEBUG) { + // System.out.println(" got " + result + "; return NOT_FOUND term=" + + // brToString(term)); + // } + return false; + } + } else { + // Follow this arc + arc = nextArc; + term.setByteAt(targetUpto, (byte) targetLabel); + // Aggregate output as we go: + assert arc.output() != null; + if (arc.output() != Lucene40BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + } + + // if (DEBUG) { + // System.out.println(" index: follow label=" + toHex(target.bytes[target.offset + + // targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput); + // } + targetUpto++; + + if (arc.isFinal()) { + // if (DEBUG) System.out.println(" arc is final!"); + currentFrame = + pushFrame( + arc, + Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), + targetUpto); + // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + + // currentFrame.hasTerms); + } + } + } + + // validIndexPrefix = targetUpto; + validIndexPrefix = currentFrame.prefix; + + currentFrame.scanToFloorFrame(target); + + // Target term is entirely contained in the index: + if (!currentFrame.hasTerms) { + termExists = false; + term.setLength(targetUpto); + // if (DEBUG) { + // System.out.println(" FAST NOT_FOUND term=" + brToString(term)); + // } + return false; + } + + currentFrame.loadBlock(); + + final SeekStatus result = currentFrame.scanToTerm(target, true); + if (result == SeekStatus.FOUND) { + // if (DEBUG) { + // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term); + // } + return true; + } else { + // if (DEBUG) { + // System.out.println(" got result " + result + "; return NOT_FOUND term=" + + // term.utf8ToString()); + // } + + return false; + } + } + + @Override + public SeekStatus seekCeil(BytesRef target) throws IOException { + + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + term.grow(1 + target.length); + + assert clearEOF(); + + // if (DEBUG) { + // System.out.println("\nBTTR.seekCeil seg=" + fr.parent.segment + " target=" + + // fr.fieldInfo.name + ":" + brToString(target) + " " + target + " current=" + brToString(term) + // + " (exists?=" + termExists + ") validIndexPrefix= " + validIndexPrefix); + // printSeekState(System.out); + // } + + FST.Arc arc; + int targetUpto; + BytesRef output; + + targetBeforeCurrentLength = currentFrame.ord; + + if (currentFrame != staticFrame) { + + // We are already seek'd; find the common + // prefix of new seek term vs current term and + // re-use the corresponding seek state. For + // example, if app first seeks to foobar, then + // seeks to foobaz, we can re-use the seek state + // for the first 5 bytes. + + // if (DEBUG) { + // System.out.println(" re-use current seek state validIndexPrefix=" + validIndexPrefix); + // } + + arc = arcs[0]; + assert arc.isFinal(); + output = arc.output(); + targetUpto = 0; + + SegmentTermsEnumFrame lastFrame = stack[0]; + assert validIndexPrefix <= term.length(); + + final int targetLimit = Math.min(target.length, validIndexPrefix); + + int cmp = 0; + + // TODO: we should write our vLong backwards (MSB + // first) to get better sharing from the FST + + // First compare up to valid seek frames: + while (targetUpto < targetLimit) { + cmp = (term.byteAt(targetUpto) & 0xFF) - (target.bytes[target.offset + targetUpto] & 0xFF); + // if (DEBUG) { + // System.out.println(" cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + + // ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " + // vs termLabel=" + (char) (term.byteAt(targetUpto)) + ")" + " arc.output=" + arc.output + + // " output=" + output); + // } + if (cmp != 0) { + break; + } + arc = arcs[1 + targetUpto]; + assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF) + : "arc.label=" + + (char) arc.label() + + " targetLabel=" + + (char) (target.bytes[target.offset + targetUpto] & 0xFF); + // TODO: we could save the outputs in local + // byte[][] instead of making new objs ever + // seek; but, often the FST doesn't have any + // shared bytes (but this could change if we + // reverse vLong byte order) + if (arc.output() != Lucene40BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + } + if (arc.isFinal()) { + lastFrame = stack[1 + lastFrame.ord]; + } + targetUpto++; + } + + if (cmp == 0) { + final int targetUptoMid = targetUpto; + // Second compare the rest of the term, but + // don't save arc/output/frame: + final int targetLimit2 = Math.min(target.length, term.length()); + while (targetUpto < targetLimit2) { + cmp = + (term.byteAt(targetUpto) & 0xFF) - (target.bytes[target.offset + targetUpto] & 0xFF); + // if (DEBUG) { + // System.out.println(" cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + // + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + // + " vs termLabel=" + (char) (term.byteAt(targetUpto)) + ")"); + // } + if (cmp != 0) { + break; + } + targetUpto++; + } + + if (cmp == 0) { + cmp = term.length() - target.length; + } + targetUpto = targetUptoMid; + } + + if (cmp < 0) { + // Common case: target term is after current + // term, ie, app is seeking multiple terms + // in sorted order + // if (DEBUG) { + // System.out.println(" target is after current (shares prefixLen=" + targetUpto + "); + // clear frame.scanned ord=" + lastFrame.ord); + // } + currentFrame = lastFrame; + + } else if (cmp > 0) { + // Uncommon case: target term + // is before current term; this means we can + // keep the currentFrame but we must rewind it + // (so we scan from the start) + targetBeforeCurrentLength = 0; + // if (DEBUG) { + // System.out.println(" target is before current (shares prefixLen=" + targetUpto + "); + // rewind frame ord=" + lastFrame.ord); + // } + currentFrame = lastFrame; + currentFrame.rewind(); + } else { + // Target is exactly the same as current term + assert term.length() == target.length; + if (termExists) { + // if (DEBUG) { + // System.out.println(" target is same as current; return FOUND"); + // } + return SeekStatus.FOUND; + } else { + // if (DEBUG) { + // System.out.println(" target is same as current but term doesn't exist"); + // } + } + } + + } else { + + targetBeforeCurrentLength = -1; + arc = fr.index.getFirstArc(arcs[0]); + + // Empty string prefix must have an output (block) in the index! + assert arc.isFinal(); + assert arc.output() != null; + + // if (DEBUG) { + // System.out.println(" no seek state; push root frame"); + // } + + output = arc.output(); + + currentFrame = staticFrame; + + // term.length = 0; + targetUpto = 0; + currentFrame = + pushFrame( + arc, Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); + } + + // if (DEBUG) { + // System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " + // currentFrame.ord+1=" + currentFrame.ord + " targetBeforeCurrentLength=" + + // targetBeforeCurrentLength); + // } + + // We are done sharing the common prefix with the incoming target and where we are currently + // seek'd; now continue walking the index: + while (targetUpto < target.length) { + + final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF; + + final FST.Arc nextArc = + fr.index.findTargetArc(targetLabel, arc, getArc(1 + targetUpto), fstReader); + + if (nextArc == null) { + + // Index is exhausted + // if (DEBUG) { + // System.out.println(" index: index exhausted label=" + ((char) targetLabel) + " " + + // targetLabel); + // } + + validIndexPrefix = currentFrame.prefix; + // validIndexPrefix = targetUpto; + + currentFrame.scanToFloorFrame(target); + + currentFrame.loadBlock(); + + // if (DEBUG) System.out.println(" now scanToTerm"); + final SeekStatus result = currentFrame.scanToTerm(target, false); + if (result == SeekStatus.END) { + term.copyBytes(target); + termExists = false; + + if (next() != null) { + // if (DEBUG) { + // System.out.println(" return NOT_FOUND term=" + brToString(term)); + // } + return SeekStatus.NOT_FOUND; + } else { + // if (DEBUG) { + // System.out.println(" return END"); + // } + return SeekStatus.END; + } + } else { + // if (DEBUG) { + // System.out.println(" return " + result + " term=" + brToString(term)); + // } + return result; + } + } else { + // Follow this arc + term.setByteAt(targetUpto, (byte) targetLabel); + arc = nextArc; + // Aggregate output as we go: + assert arc.output() != null; + if (arc.output() != Lucene40BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + } + + // if (DEBUG) { + // System.out.println(" index: follow label=" + (target.bytes[target.offset + + // targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput); + // } + targetUpto++; + + if (arc.isFinal()) { + // if (DEBUG) System.out.println(" arc is final!"); + currentFrame = + pushFrame( + arc, + Lucene40BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), + targetUpto); + // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + + // currentFrame.hasTerms); + } + } + } + + // validIndexPrefix = targetUpto; + validIndexPrefix = currentFrame.prefix; + + currentFrame.scanToFloorFrame(target); + + currentFrame.loadBlock(); + + final SeekStatus result = currentFrame.scanToTerm(target, false); + + if (result == SeekStatus.END) { + term.copyBytes(target); + termExists = false; + if (next() != null) { + // if (DEBUG) { + // System.out.println(" return NOT_FOUND term=" + term.get().utf8ToString() + " " + term); + // } + return SeekStatus.NOT_FOUND; + } else { + // if (DEBUG) { + // System.out.println(" return END"); + // } + return SeekStatus.END; + } + } else { + return result; + } + } + + @SuppressWarnings("unused") + private void printSeekState(PrintStream out) throws IOException { + if (currentFrame == staticFrame) { + out.println(" no prior seek"); + } else { + out.println(" prior seek state:"); + int ord = 0; + boolean isSeekFrame = true; + while (true) { + SegmentTermsEnumFrame f = getFrame(ord); + assert f != null; + final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefix); + if (f.nextEnt == -1) { + out.println( + " frame " + + (isSeekFrame ? "(seek)" : "(next)") + + " ord=" + + ord + + " fp=" + + f.fp + + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + + " prefixLen=" + + f.prefix + + " prefix=" + + prefix + + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + + " hasTerms=" + + f.hasTerms + + " isFloor=" + + f.isFloor + + " code=" + + ((f.fp << Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + + (f.hasTerms ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + + (f.isFloor ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + + " isLastInFloor=" + + f.isLastInFloor + + " mdUpto=" + + f.metaDataUpto + + " tbOrd=" + + f.getTermBlockOrd()); + } else { + out.println( + " frame " + + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + + " ord=" + + ord + + " fp=" + + f.fp + + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + + " prefixLen=" + + f.prefix + + " prefix=" + + prefix + + " nextEnt=" + + f.nextEnt + + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + + " hasTerms=" + + f.hasTerms + + " isFloor=" + + f.isFloor + + " code=" + + ((f.fp << Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + + (f.hasTerms ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + + (f.isFloor ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + + " lastSubFP=" + + f.lastSubFP + + " isLastInFloor=" + + f.isLastInFloor + + " mdUpto=" + + f.metaDataUpto + + " tbOrd=" + + f.getTermBlockOrd()); + } + if (fr.index != null) { + assert !isSeekFrame || f.arc != null : "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc; + if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix - 1) & 0xFF)) { + out.println( + " broken seek state: arc.label=" + + (char) f.arc.label() + + " vs term byte=" + + (char) (term.byteAt(f.prefix - 1) & 0xFF)); + throw new RuntimeException("seek state is broken"); + } + BytesRef output = Util.get(fr.index, prefix); + if (output == null) { + out.println(" broken seek state: prefix is not final in index"); + throw new RuntimeException("seek state is broken"); + } else if (isSeekFrame && !f.isFloor) { + final ByteArrayDataInput reader = + new ByteArrayDataInput(output.bytes, output.offset, output.length); + final long codeOrig = reader.readVLong(); + final long code = + (f.fp << Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + | (f.hasTerms ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + | (f.isFloor ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); + if (codeOrig != code) { + out.println( + " broken seek state: output code=" + + codeOrig + + " doesn't match frame code=" + + code); + throw new RuntimeException("seek state is broken"); + } + } + } + if (f == currentFrame) { + break; + } + if (f.prefix == validIndexPrefix) { + isSeekFrame = false; + } + ord++; + } + } + } + + /* Decodes only the term bytes of the next term. If caller then asks for + metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily) + decode all metadata up to the current term. */ + @Override + public BytesRef next() throws IOException { + if (in == null) { + // Fresh TermsEnum; seek to first term: + final FST.Arc arc; + if (fr.index != null) { + arc = fr.index.getFirstArc(arcs[0]); + // Empty string prefix must have an output in the index! + assert arc.isFinal(); + } else { + arc = null; + } + currentFrame = pushFrame(arc, fr.rootCode, 0); + currentFrame.loadBlock(); + } + + targetBeforeCurrentLength = currentFrame.ord; + + assert !eof; + // if (DEBUG) { + // System.out.println("\nBTTR.next seg=" + fr.parent.segment + " term=" + brToString(term) + " + // termExists?=" + termExists + " field=" + fr.fieldInfo.name + " termBlockOrd=" + + // currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix); + // printSeekState(System.out); + // } + + if (currentFrame == staticFrame) { + // If seek was previously called and the term was + // cached, or seek(TermState) was called, usually + // caller is just going to pull a D/&PEnum or get + // docFreq, etc. But, if they then call next(), + // this method catches up all internal state so next() + // works properly: + // if (DEBUG) System.out.println(" re-seek to pending term=" + term.utf8ToString() + " " + + // term); + final boolean result = seekExact(term.get()); + assert result; + } + + // Pop finished blocks + while (currentFrame.nextEnt == currentFrame.entCount) { + if (!currentFrame.isLastInFloor) { + // Advance to next floor block + currentFrame.loadNextFloorBlock(); + break; + } else { + // if (DEBUG) System.out.println(" pop frame"); + if (currentFrame.ord == 0) { + // if (DEBUG) System.out.println(" return null"); + assert setEOF(); + term.clear(); + validIndexPrefix = 0; + currentFrame.rewind(); + termExists = false; + return null; + } + final long lastFP = currentFrame.fpOrig; + currentFrame = stack[currentFrame.ord - 1]; + + if (currentFrame.nextEnt == -1 || currentFrame.lastSubFP != lastFP) { + // We popped into a frame that's not loaded + // yet or not scan'd to the right entry + currentFrame.scanToFloorFrame(term.get()); + currentFrame.loadBlock(); + currentFrame.scanToSubBlock(lastFP); + } + + // Note that the seek state (last seek) has been + // invalidated beyond this depth + validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix); + // if (DEBUG) { + // System.out.println(" reset validIndexPrefix=" + validIndexPrefix); + // } + } + } + + while (true) { + if (currentFrame.next()) { + // Push to new block: + // if (DEBUG) System.out.println(" push frame"); + currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length()); + // This is a "next" frame -- even if it's + // floor'd we must pretend it isn't so we don't + // try to scan to the right floor frame: + currentFrame.loadBlock(); + } else { + // if (DEBUG) System.out.println(" return term=" + brToString(term) + " currentFrame.ord=" + // + currentFrame.ord); + return term.get(); + } + } + } + + @Override + public BytesRef term() { + assert !eof; + return term.get(); + } + + @Override + public int docFreq() throws IOException { + assert !eof; + // if (DEBUG) System.out.println("BTR.docFreq"); + currentFrame.decodeMetaData(); + // if (DEBUG) System.out.println(" return " + currentFrame.state.docFreq); + return currentFrame.state.docFreq; + } + + @Override + public long totalTermFreq() throws IOException { + assert !eof; + currentFrame.decodeMetaData(); + return currentFrame.state.totalTermFreq; + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + assert !eof; + // if (DEBUG) { + // System.out.println("BTTR.docs seg=" + segment); + // } + currentFrame.decodeMetaData(); + // if (DEBUG) { + // System.out.println(" state=" + currentFrame.state); + // } + return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, reuse, flags); + } + + @Override + public ImpactsEnum impacts(int flags) throws IOException { + assert !eof; + // if (DEBUG) { + // System.out.println("BTTR.docs seg=" + segment); + // } + currentFrame.decodeMetaData(); + // if (DEBUG) { + // System.out.println(" state=" + currentFrame.state); + // } + return fr.parent.postingsReader.impacts(fr.fieldInfo, currentFrame.state, flags); + } + + @Override + public void seekExact(BytesRef target, TermState otherState) { + // if (DEBUG) { + // System.out.println("BTTR.seekExact termState seg=" + segment + " target=" + + // target.utf8ToString() + " " + target + " state=" + otherState); + // } + assert clearEOF(); + if (target.compareTo(term.get()) != 0 || !termExists) { + assert otherState != null && otherState instanceof BlockTermState; + currentFrame = staticFrame; + currentFrame.state.copyFrom(otherState); + term.copyBytes(target); + currentFrame.metaDataUpto = currentFrame.getTermBlockOrd(); + assert currentFrame.metaDataUpto > 0; + validIndexPrefix = 0; + } else { + // if (DEBUG) { + // System.out.println(" skip seek: already on target state=" + currentFrame.state); + // } + } + } + + @Override + public TermState termState() throws IOException { + assert !eof; + currentFrame.decodeMetaData(); + TermState ts = currentFrame.state.clone(); + // if (DEBUG) System.out.println("BTTR.termState seg=" + segment + " state=" + ts); + return ts; + } + + @Override + public void seekExact(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnumFrame.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnumFrame.java index eb91b04578a..afe902fa3a8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/SegmentTermsEnumFrame.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.backward_codecs.lucene40.blocktree; import java.io.IOException; import java.util.Arrays; @@ -102,7 +102,7 @@ final class SegmentTermsEnumFrame { this.state = ste.fr.parent.postingsReader.newTermState(); this.state.totalTermFreq = -1; this.version = ste.fr.parent.version; - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { suffixLengthBytes = new byte[32]; suffixLengthsReader = new ByteArrayDataInput(); } else { @@ -179,7 +179,7 @@ final class SegmentTermsEnumFrame { final long startSuffixFP = ste.in.getFilePointer(); // term suffixes: - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { final long codeL = ste.in.readVLong(); isLeafBlock = (codeL & 0x04) != 0; final int numSuffixBytes = (int) (codeL >>> 3); @@ -471,7 +471,7 @@ final class SegmentTermsEnumFrame { // TODO: if docFreq were bulk decoded we could // just skipN here: - if (version >= BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { + if (version >= Lucene40BlockTreeTermsReader.VERSION_COMPRESSED_SUFFIXES) { if (statsSingletonRunLength > 0) { state.docFreq = 1; state.totalTermFreq = 1; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Stats.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Stats.java new file mode 100644 index 00000000000..9c0a819694b --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Stats.java @@ -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); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/package-info.java similarity index 89% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/package-info.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/package-info.java index dec1f2cc52f..a34b7aa345d 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package-info.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/package-info.java @@ -23,6 +23,6 @@ * structure. It allows you to plug in your own {@link org.apache.lucene.codecs.PostingsWriterBase} * to implement the postings. * - *

See {@link org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter} for the file format. + *

See {@code BlockTreeTermsWriter} for the file format. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.backward_codecs.lucene40.blocktree; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50PostingsFormat.java index 325492bfe08..27191a76749 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50PostingsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50PostingsFormat.java @@ -17,6 +17,7 @@ package org.apache.lucene.backward_codecs.lucene50; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsReader; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; 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.PostingsFormat; 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.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -93,7 +92,7 @@ import org.apache.lucene.util.packed.PackedInts; *

Term Dictionary *

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 BlockTreeTermsWriter} for more details on the + * .doc, .pos, and .pay files. See {@code BlockTreeTermsWriter} for more details on the * format. *

NOTE: The term dictionary can plug into different postings implementations: the postings * writer/reader are actually responsible for encoding and decoding the PostingsHeader and @@ -148,7 +147,7 @@ import org.apache.lucene.util.packed.PackedInts; *

*
Term Index *

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. *

* * @@ -387,7 +386,7 @@ public class Lucene50PostingsFormat extends PostingsFormat { PostingsReaderBase postingsReader = new Lucene50PostingsReader(state); boolean success = false; try { - FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state); + FieldsProducer ret = new Lucene40BlockTreeTermsReader(postingsReader, state); success = true; return ret; } finally { diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForDeltaUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForDeltaUtil.java new file mode 100644 index 00000000000..dd5f1f7a6a3 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForDeltaUtil.java @@ -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)); + } + } +} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForUtil.java new file mode 100644 index 00000000000..6350c914b46 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/ForUtil.java @@ -0,0 +1,1150 @@ +// 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.backward_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; + } + } + + private static final long[] MASKS8 = new long[8]; + private static final long[] MASKS16 = new long[16]; + private static final long[] MASKS32 = new long[32]; + + static { + for (int i = 0; i < 8; ++i) { + MASKS8[i] = mask8(i); + } + for (int i = 0; i < 16; ++i) { + MASKS16[i] = mask16(i); + } + for (int i = 0; i < 32; ++i) { + MASKS32[i] = mask32(i); + } + } + // 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 + private static final long MASK8_1 = MASKS8[1]; + private static final long MASK8_2 = MASKS8[2]; + private static final long MASK8_3 = MASKS8[3]; + private static final long MASK8_4 = MASKS8[4]; + private static final long MASK8_5 = MASKS8[5]; + private static final long MASK8_6 = MASKS8[6]; + private static final long MASK8_7 = MASKS8[7]; + private static final long MASK16_1 = MASKS16[1]; + private static final long MASK16_2 = MASKS16[2]; + private static final long MASK16_3 = MASKS16[3]; + private static final long MASK16_4 = MASKS16[4]; + private static final long MASK16_5 = MASKS16[5]; + private static final long MASK16_6 = MASKS16[6]; + private static final long MASK16_7 = MASKS16[7]; + private static final long MASK16_9 = MASKS16[9]; + private static final long MASK16_10 = MASKS16[10]; + private static final long MASK16_11 = MASKS16[11]; + private static final long MASK16_12 = MASKS16[12]; + private static final long MASK16_13 = MASKS16[13]; + private static final long MASK16_14 = MASKS16[14]; + private static final long MASK16_15 = MASKS16[15]; + private static final long MASK32_1 = MASKS32[1]; + private static final long MASK32_2 = MASKS32[2]; + private static final long MASK32_3 = MASKS32[3]; + private static final long MASK32_4 = MASKS32[4]; + private static final long MASK32_5 = MASKS32[5]; + private static final long MASK32_6 = MASKS32[6]; + private static final long MASK32_7 = MASKS32[7]; + private static final long MASK32_8 = MASKS32[8]; + private static final long MASK32_9 = MASKS32[9]; + private static final long MASK32_10 = MASKS32[10]; + private static final long MASK32_11 = MASKS32[11]; + private static final long MASK32_12 = MASKS32[12]; + private static final long MASK32_13 = MASKS32[13]; + private static final long MASK32_14 = MASKS32[14]; + private static final long MASK32_15 = MASKS32[15]; + private static final long MASK32_17 = MASKS32[17]; + private static final long MASK32_18 = MASKS32[18]; + private static final long MASK32_19 = MASKS32[19]; + private static final long MASK32_20 = MASKS32[20]; + private static final long MASK32_21 = MASKS32[21]; + private static final long MASK32_22 = MASKS32[22]; + private static final long MASK32_23 = MASKS32[23]; + private static final long MASK32_24 = MASKS32[24]; + + /** Decode 128 integers into {@code longs}. */ + void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + expand8(longs); + break; + case 2: + decode2(in, tmp, longs); + expand8(longs); + break; + case 3: + decode3(in, tmp, longs); + expand8(longs); + break; + case 4: + decode4(in, tmp, longs); + expand8(longs); + break; + case 5: + decode5(in, tmp, longs); + expand8(longs); + break; + case 6: + decode6(in, tmp, longs); + expand8(longs); + break; + case 7: + decode7(in, tmp, longs); + expand8(longs); + break; + case 8: + decode8(in, tmp, longs); + expand8(longs); + break; + case 9: + decode9(in, tmp, longs); + expand16(longs); + break; + case 10: + decode10(in, tmp, longs); + expand16(longs); + break; + case 11: + decode11(in, tmp, longs); + expand16(longs); + break; + case 12: + decode12(in, tmp, longs); + expand16(longs); + break; + case 13: + decode13(in, tmp, longs); + expand16(longs); + break; + case 14: + decode14(in, tmp, longs); + expand16(longs); + break; + case 15: + decode15(in, tmp, longs); + expand16(longs); + break; + case 16: + decode16(in, tmp, longs); + expand16(longs); + break; + case 17: + decode17(in, tmp, longs); + expand32(longs); + break; + case 18: + decode18(in, tmp, longs); + expand32(longs); + break; + case 19: + decode19(in, tmp, longs); + expand32(longs); + break; + case 20: + decode20(in, tmp, longs); + expand32(longs); + break; + case 21: + decode21(in, tmp, longs); + expand32(longs); + break; + case 22: + decode22(in, tmp, longs); + expand32(longs); + break; + case 23: + decode23(in, tmp, longs); + expand32(longs); + break; + case 24: + decode24(in, tmp, longs); + expand32(longs); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + expand32(longs); + break; + } + } + + /** Delta-decode 128 integers into {@code longs}. */ + void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) + throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + prefixSum8(longs, base); + break; + case 2: + decode2(in, tmp, longs); + prefixSum8(longs, base); + break; + case 3: + decode3(in, tmp, longs); + prefixSum8(longs, base); + break; + case 4: + decode4(in, tmp, longs); + prefixSum8(longs, base); + break; + case 5: + decode5(in, tmp, longs); + prefixSum8(longs, base); + break; + case 6: + decode6(in, tmp, longs); + prefixSum8(longs, base); + break; + case 7: + decode7(in, tmp, longs); + prefixSum8(longs, base); + break; + case 8: + decode8(in, tmp, longs); + prefixSum8(longs, base); + break; + case 9: + decode9(in, tmp, longs); + prefixSum16(longs, base); + break; + case 10: + decode10(in, tmp, longs); + prefixSum16(longs, base); + break; + case 11: + decode11(in, tmp, longs); + prefixSum16(longs, base); + break; + case 12: + decode12(in, tmp, longs); + prefixSum16(longs, base); + break; + case 13: + decode13(in, tmp, longs); + prefixSum16(longs, base); + break; + case 14: + decode14(in, tmp, longs); + prefixSum16(longs, base); + break; + case 15: + decode15(in, tmp, longs); + prefixSum16(longs, base); + break; + case 16: + decode16(in, tmp, longs); + prefixSum16(longs, base); + break; + case 17: + decode17(in, tmp, longs); + prefixSum32(longs, base); + break; + case 18: + decode18(in, tmp, longs); + prefixSum32(longs, base); + break; + case 19: + decode19(in, tmp, longs); + prefixSum32(longs, base); + break; + case 20: + decode20(in, tmp, longs); + prefixSum32(longs, base); + break; + case 21: + decode21(in, tmp, longs); + prefixSum32(longs, base); + break; + case 22: + decode22(in, tmp, longs); + prefixSum32(longs, base); + break; + case 23: + decode23(in, tmp, longs); + prefixSum32(longs, base); + break; + case 24: + decode24(in, tmp, longs); + prefixSum32(longs, base); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + prefixSum32(longs, base); + break; + } + } + + private static void decode1(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 2); + shiftLongs(tmp, 2, longs, 0, 7, MASK8_1); + shiftLongs(tmp, 2, longs, 2, 6, MASK8_1); + shiftLongs(tmp, 2, longs, 4, 5, MASK8_1); + shiftLongs(tmp, 2, longs, 6, 4, MASK8_1); + shiftLongs(tmp, 2, longs, 8, 3, MASK8_1); + shiftLongs(tmp, 2, longs, 10, 2, MASK8_1); + shiftLongs(tmp, 2, longs, 12, 1, MASK8_1); + shiftLongs(tmp, 2, longs, 14, 0, MASK8_1); + } + + private static void decode2(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 4); + shiftLongs(tmp, 4, longs, 0, 6, MASK8_2); + shiftLongs(tmp, 4, longs, 4, 4, MASK8_2); + shiftLongs(tmp, 4, longs, 8, 2, MASK8_2); + shiftLongs(tmp, 4, longs, 12, 0, MASK8_2); + } + + private static void decode3(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 6); + shiftLongs(tmp, 6, longs, 0, 5, MASK8_3); + shiftLongs(tmp, 6, longs, 6, 2, MASK8_3); + for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 2; ++iter, tmpIdx += 3, longsIdx += 2) { + long l0 = (tmp[tmpIdx + 0] & MASK8_2) << 1; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 2; + l1 |= (tmp[tmpIdx + 2] & MASK8_2) << 0; + longs[longsIdx + 1] = l1; + } + } + + private static void decode4(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 8); + shiftLongs(tmp, 8, longs, 0, 4, MASK8_4); + shiftLongs(tmp, 8, longs, 8, 0, MASK8_4); + } + + private static void decode5(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 10); + shiftLongs(tmp, 10, longs, 0, 3, MASK8_5); + for (int iter = 0, tmpIdx = 0, longsIdx = 10; iter < 2; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK8_3) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 4; + l1 |= (tmp[tmpIdx + 2] & MASK8_3) << 1; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK8_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK8_2) << 3; + l2 |= (tmp[tmpIdx + 4] & MASK8_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode6(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 12); + shiftLongs(tmp, 12, longs, 0, 2, MASK8_6); + shiftLongs(tmp, 12, tmp, 0, 0, MASK8_2); + for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 4; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 4; + l0 |= tmp[tmpIdx + 1] << 2; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode7(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 14); + shiftLongs(tmp, 14, longs, 0, 1, MASK8_7); + shiftLongs(tmp, 14, tmp, 0, 0, MASK8_1); + for (int iter = 0, tmpIdx = 0, longsIdx = 14; iter < 2; ++iter, tmpIdx += 7, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 6; + l0 |= tmp[tmpIdx + 1] << 5; + l0 |= tmp[tmpIdx + 2] << 4; + l0 |= tmp[tmpIdx + 3] << 3; + l0 |= tmp[tmpIdx + 4] << 2; + l0 |= tmp[tmpIdx + 5] << 1; + l0 |= tmp[tmpIdx + 6] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode8(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(longs, 0, 16); + } + + private static void decode9(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 18); + shiftLongs(tmp, 18, longs, 0, 7, MASK16_9); + for (int iter = 0, tmpIdx = 0, longsIdx = 18; iter < 2; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK16_7) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_5) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 3) & MASK16_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK16_1) << 8; + l3 |= (tmp[tmpIdx + 4] & MASK16_7) << 1; + l3 |= (tmp[tmpIdx + 5] >>> 6) & MASK16_1; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK16_6) << 3; + l4 |= (tmp[tmpIdx + 6] >>> 4) & MASK16_3; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK16_4) << 5; + l5 |= (tmp[tmpIdx + 7] >>> 2) & MASK16_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK16_2) << 7; + l6 |= (tmp[tmpIdx + 8] & MASK16_7) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode10(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 20); + shiftLongs(tmp, 20, longs, 0, 6, MASK16_10); + for (int iter = 0, tmpIdx = 0, longsIdx = 20; iter < 4; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_6) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_2) << 8; + l1 |= (tmp[tmpIdx + 2] & MASK16_6) << 2; + l1 |= (tmp[tmpIdx + 3] >>> 4) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK16_4) << 6; + l2 |= (tmp[tmpIdx + 4] & MASK16_6) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode11(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 22); + shiftLongs(tmp, 22, longs, 0, 5, MASK16_11); + for (int iter = 0, tmpIdx = 0, longsIdx = 22; iter < 2; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK16_5) << 6; + l0 |= (tmp[tmpIdx + 1] & MASK16_5) << 1; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK16_4) << 7; + l1 |= (tmp[tmpIdx + 3] & MASK16_5) << 2; + l1 |= (tmp[tmpIdx + 4] >>> 3) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK16_3) << 8; + l2 |= (tmp[tmpIdx + 5] & MASK16_5) << 3; + l2 |= (tmp[tmpIdx + 6] >>> 2) & MASK16_3; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK16_2) << 9; + l3 |= (tmp[tmpIdx + 7] & MASK16_5) << 4; + l3 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_4; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK16_1) << 10; + l4 |= (tmp[tmpIdx + 9] & MASK16_5) << 5; + l4 |= (tmp[tmpIdx + 10] & MASK16_5) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode12(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 24); + shiftLongs(tmp, 24, longs, 0, 4, MASK16_12); + shiftLongs(tmp, 24, tmp, 0, 0, MASK16_4); + for (int iter = 0, tmpIdx = 0, longsIdx = 24; iter < 8; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 8; + l0 |= tmp[tmpIdx + 1] << 4; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode13(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 26); + shiftLongs(tmp, 26, longs, 0, 3, MASK16_13); + for (int iter = 0, tmpIdx = 0, longsIdx = 26; iter < 2; ++iter, tmpIdx += 13, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_3) << 10; + l0 |= (tmp[tmpIdx + 1] & MASK16_3) << 7; + l0 |= (tmp[tmpIdx + 2] & MASK16_3) << 4; + l0 |= (tmp[tmpIdx + 3] & MASK16_3) << 1; + l0 |= (tmp[tmpIdx + 4] >>> 2) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 4] & MASK16_2) << 11; + l1 |= (tmp[tmpIdx + 5] & MASK16_3) << 8; + l1 |= (tmp[tmpIdx + 6] & MASK16_3) << 5; + l1 |= (tmp[tmpIdx + 7] & MASK16_3) << 2; + l1 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 8] & MASK16_1) << 12; + l2 |= (tmp[tmpIdx + 9] & MASK16_3) << 9; + l2 |= (tmp[tmpIdx + 10] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 11] & MASK16_3) << 3; + l2 |= (tmp[tmpIdx + 12] & MASK16_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode14(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 28); + shiftLongs(tmp, 28, longs, 0, 2, MASK16_14); + shiftLongs(tmp, 28, tmp, 0, 0, MASK16_2); + for (int iter = 0, tmpIdx = 0, longsIdx = 28; iter < 4; ++iter, tmpIdx += 7, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 12; + l0 |= tmp[tmpIdx + 1] << 10; + l0 |= tmp[tmpIdx + 2] << 8; + l0 |= tmp[tmpIdx + 3] << 6; + l0 |= tmp[tmpIdx + 4] << 4; + l0 |= tmp[tmpIdx + 5] << 2; + l0 |= tmp[tmpIdx + 6] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode15(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 30); + shiftLongs(tmp, 30, longs, 0, 1, MASK16_15); + shiftLongs(tmp, 30, tmp, 0, 0, MASK16_1); + for (int iter = 0, tmpIdx = 0, longsIdx = 30; iter < 2; ++iter, tmpIdx += 15, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 14; + l0 |= tmp[tmpIdx + 1] << 13; + l0 |= tmp[tmpIdx + 2] << 12; + l0 |= tmp[tmpIdx + 3] << 11; + l0 |= tmp[tmpIdx + 4] << 10; + l0 |= tmp[tmpIdx + 5] << 9; + l0 |= tmp[tmpIdx + 6] << 8; + l0 |= tmp[tmpIdx + 7] << 7; + l0 |= tmp[tmpIdx + 8] << 6; + l0 |= tmp[tmpIdx + 9] << 5; + l0 |= tmp[tmpIdx + 10] << 4; + l0 |= tmp[tmpIdx + 11] << 3; + l0 |= tmp[tmpIdx + 12] << 2; + l0 |= tmp[tmpIdx + 13] << 1; + l0 |= tmp[tmpIdx + 14] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode16(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(longs, 0, 32); + } + + private static void decode17(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 34); + shiftLongs(tmp, 34, longs, 0, 15, MASK32_17); + for (int iter = 0, tmpIdx = 0, longsIdx = 34; iter < 2; ++iter, tmpIdx += 17, longsIdx += 15) { + long l0 = (tmp[tmpIdx + 0] & MASK32_15) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_13) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 11) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_11) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 9) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_9) << 8; + l3 |= (tmp[tmpIdx + 4] >>> 7) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 4] & MASK32_7) << 10; + l4 |= (tmp[tmpIdx + 5] >>> 5) & MASK32_10; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 5] & MASK32_5) << 12; + l5 |= (tmp[tmpIdx + 6] >>> 3) & MASK32_12; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 6] & MASK32_3) << 14; + l6 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_14; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 7] & MASK32_1) << 16; + l7 |= (tmp[tmpIdx + 8] & MASK32_15) << 1; + l7 |= (tmp[tmpIdx + 9] >>> 14) & MASK32_1; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 9] & MASK32_14) << 3; + l8 |= (tmp[tmpIdx + 10] >>> 12) & MASK32_3; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 10] & MASK32_12) << 5; + l9 |= (tmp[tmpIdx + 11] >>> 10) & MASK32_5; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 11] & MASK32_10) << 7; + l10 |= (tmp[tmpIdx + 12] >>> 8) & MASK32_7; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 12] & MASK32_8) << 9; + l11 |= (tmp[tmpIdx + 13] >>> 6) & MASK32_9; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 13] & MASK32_6) << 11; + l12 |= (tmp[tmpIdx + 14] >>> 4) & MASK32_11; + longs[longsIdx + 12] = l12; + long l13 = (tmp[tmpIdx + 14] & MASK32_4) << 13; + l13 |= (tmp[tmpIdx + 15] >>> 2) & MASK32_13; + longs[longsIdx + 13] = l13; + long l14 = (tmp[tmpIdx + 15] & MASK32_2) << 15; + l14 |= (tmp[tmpIdx + 16] & MASK32_15) << 0; + longs[longsIdx + 14] = l14; + } + } + + private static void decode18(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 36); + shiftLongs(tmp, 36, longs, 0, 14, MASK32_18); + for (int iter = 0, tmpIdx = 0, longsIdx = 36; iter < 4; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK32_14) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_10) << 8; + l1 |= (tmp[tmpIdx + 2] >>> 6) & MASK32_8; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_6) << 12; + l2 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_12; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_2) << 16; + l3 |= (tmp[tmpIdx + 4] & MASK32_14) << 2; + l3 |= (tmp[tmpIdx + 5] >>> 12) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_12) << 6; + l4 |= (tmp[tmpIdx + 6] >>> 8) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK32_8) << 10; + l5 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK32_4) << 14; + l6 |= (tmp[tmpIdx + 8] & MASK32_14) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode19(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 38); + shiftLongs(tmp, 38, longs, 0, 13, MASK32_19); + for (int iter = 0, tmpIdx = 0, longsIdx = 38; iter < 2; ++iter, tmpIdx += 19, longsIdx += 13) { + long l0 = (tmp[tmpIdx + 0] & MASK32_13) << 6; + l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_7) << 12; + l1 |= (tmp[tmpIdx + 2] >>> 1) & MASK32_12; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_1) << 18; + l2 |= (tmp[tmpIdx + 3] & MASK32_13) << 5; + l2 |= (tmp[tmpIdx + 4] >>> 8) & MASK32_5; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 4] & MASK32_8) << 11; + l3 |= (tmp[tmpIdx + 5] >>> 2) & MASK32_11; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_2) << 17; + l4 |= (tmp[tmpIdx + 6] & MASK32_13) << 4; + l4 |= (tmp[tmpIdx + 7] >>> 9) & MASK32_4; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 7] & MASK32_9) << 10; + l5 |= (tmp[tmpIdx + 8] >>> 3) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 8] & MASK32_3) << 16; + l6 |= (tmp[tmpIdx + 9] & MASK32_13) << 3; + l6 |= (tmp[tmpIdx + 10] >>> 10) & MASK32_3; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 10] & MASK32_10) << 9; + l7 |= (tmp[tmpIdx + 11] >>> 4) & MASK32_9; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 11] & MASK32_4) << 15; + l8 |= (tmp[tmpIdx + 12] & MASK32_13) << 2; + l8 |= (tmp[tmpIdx + 13] >>> 11) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 13] & MASK32_11) << 8; + l9 |= (tmp[tmpIdx + 14] >>> 5) & MASK32_8; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 14] & MASK32_5) << 14; + l10 |= (tmp[tmpIdx + 15] & MASK32_13) << 1; + l10 |= (tmp[tmpIdx + 16] >>> 12) & MASK32_1; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 16] & MASK32_12) << 7; + l11 |= (tmp[tmpIdx + 17] >>> 6) & MASK32_7; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 17] & MASK32_6) << 13; + l12 |= (tmp[tmpIdx + 18] & MASK32_13) << 0; + longs[longsIdx + 12] = l12; + } + } + + private static void decode20(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 40); + shiftLongs(tmp, 40, longs, 0, 12, MASK32_20); + for (int iter = 0, tmpIdx = 0, longsIdx = 40; iter < 8; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK32_12) << 8; + l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_4) << 16; + l1 |= (tmp[tmpIdx + 2] & MASK32_12) << 4; + l1 |= (tmp[tmpIdx + 3] >>> 8) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_8) << 12; + l2 |= (tmp[tmpIdx + 4] & MASK32_12) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode21(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 42); + shiftLongs(tmp, 42, longs, 0, 11, MASK32_21); + for (int iter = 0, tmpIdx = 0, longsIdx = 42; iter < 2; ++iter, tmpIdx += 21, longsIdx += 11) { + long l0 = (tmp[tmpIdx + 0] & MASK32_11) << 10; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_1) << 20; + l1 |= (tmp[tmpIdx + 2] & MASK32_11) << 9; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_9; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_2) << 19; + l2 |= (tmp[tmpIdx + 4] & MASK32_11) << 8; + l2 |= (tmp[tmpIdx + 5] >>> 3) & MASK32_8; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 5] & MASK32_3) << 18; + l3 |= (tmp[tmpIdx + 6] & MASK32_11) << 7; + l3 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_7; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 7] & MASK32_4) << 17; + l4 |= (tmp[tmpIdx + 8] & MASK32_11) << 6; + l4 |= (tmp[tmpIdx + 9] >>> 5) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 9] & MASK32_5) << 16; + l5 |= (tmp[tmpIdx + 10] & MASK32_11) << 5; + l5 |= (tmp[tmpIdx + 11] >>> 6) & MASK32_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 11] & MASK32_6) << 15; + l6 |= (tmp[tmpIdx + 12] & MASK32_11) << 4; + l6 |= (tmp[tmpIdx + 13] >>> 7) & MASK32_4; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 13] & MASK32_7) << 14; + l7 |= (tmp[tmpIdx + 14] & MASK32_11) << 3; + l7 |= (tmp[tmpIdx + 15] >>> 8) & MASK32_3; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 15] & MASK32_8) << 13; + l8 |= (tmp[tmpIdx + 16] & MASK32_11) << 2; + l8 |= (tmp[tmpIdx + 17] >>> 9) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 17] & MASK32_9) << 12; + l9 |= (tmp[tmpIdx + 18] & MASK32_11) << 1; + l9 |= (tmp[tmpIdx + 19] >>> 10) & MASK32_1; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 19] & MASK32_10) << 11; + l10 |= (tmp[tmpIdx + 20] & MASK32_11) << 0; + longs[longsIdx + 10] = l10; + } + } + + private static void decode22(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 44); + shiftLongs(tmp, 44, longs, 0, 10, MASK32_22); + for (int iter = 0, tmpIdx = 0, longsIdx = 44; iter < 4; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK32_10) << 12; + l0 |= (tmp[tmpIdx + 1] & MASK32_10) << 2; + l0 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_8) << 14; + l1 |= (tmp[tmpIdx + 3] & MASK32_10) << 4; + l1 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK32_6) << 16; + l2 |= (tmp[tmpIdx + 5] & MASK32_10) << 6; + l2 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK32_4) << 18; + l3 |= (tmp[tmpIdx + 7] & MASK32_10) << 8; + l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK32_2) << 20; + l4 |= (tmp[tmpIdx + 9] & MASK32_10) << 10; + l4 |= (tmp[tmpIdx + 10] & MASK32_10) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode23(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 46); + shiftLongs(tmp, 46, longs, 0, 9, MASK32_23); + for (int iter = 0, tmpIdx = 0, longsIdx = 46; iter < 2; ++iter, tmpIdx += 23, longsIdx += 9) { + long l0 = (tmp[tmpIdx + 0] & MASK32_9) << 14; + l0 |= (tmp[tmpIdx + 1] & MASK32_9) << 5; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK32_5; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_4) << 19; + l1 |= (tmp[tmpIdx + 3] & MASK32_9) << 10; + l1 |= (tmp[tmpIdx + 4] & MASK32_9) << 1; + l1 |= (tmp[tmpIdx + 5] >>> 8) & MASK32_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 5] & MASK32_8) << 15; + l2 |= (tmp[tmpIdx + 6] & MASK32_9) << 6; + l2 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 7] & MASK32_3) << 20; + l3 |= (tmp[tmpIdx + 8] & MASK32_9) << 11; + l3 |= (tmp[tmpIdx + 9] & MASK32_9) << 2; + l3 |= (tmp[tmpIdx + 10] >>> 7) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 10] & MASK32_7) << 16; + l4 |= (tmp[tmpIdx + 11] & MASK32_9) << 7; + l4 |= (tmp[tmpIdx + 12] >>> 2) & MASK32_7; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 12] & MASK32_2) << 21; + l5 |= (tmp[tmpIdx + 13] & MASK32_9) << 12; + l5 |= (tmp[tmpIdx + 14] & MASK32_9) << 3; + l5 |= (tmp[tmpIdx + 15] >>> 6) & MASK32_3; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 15] & MASK32_6) << 17; + l6 |= (tmp[tmpIdx + 16] & MASK32_9) << 8; + l6 |= (tmp[tmpIdx + 17] >>> 1) & MASK32_8; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 17] & MASK32_1) << 22; + l7 |= (tmp[tmpIdx + 18] & MASK32_9) << 13; + l7 |= (tmp[tmpIdx + 19] & MASK32_9) << 4; + l7 |= (tmp[tmpIdx + 20] >>> 5) & MASK32_4; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 20] & MASK32_5) << 18; + l8 |= (tmp[tmpIdx + 21] & MASK32_9) << 9; + l8 |= (tmp[tmpIdx + 22] & MASK32_9) << 0; + longs[longsIdx + 8] = l8; + } + } + + private static void decode24(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLELongs(tmp, 0, 48); + shiftLongs(tmp, 48, longs, 0, 8, MASK32_24); + shiftLongs(tmp, 48, tmp, 0, 0, MASK32_8); + for (int iter = 0, tmpIdx = 0, longsIdx = 48; iter < 16; ++iter, tmpIdx += 3, longsIdx += 1) { + long l0 = tmp[tmpIdx + 0] << 16; + l0 |= tmp[tmpIdx + 1] << 8; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } +} diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java index c476e9fbcf4..4f9db5587f7 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84Codec.java @@ -39,7 +39,6 @@ import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; 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.PerFieldPostingsFormat; @@ -49,7 +48,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; * *

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 */ public class Lucene84Codec extends Codec { @@ -105,7 +104,7 @@ public class Lucene84Codec extends Codec { } @Override - public final PostingsFormat postingsFormat() { + public PostingsFormat postingsFormat() { return postingsFormat; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsFormat.java similarity index 94% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsFormat.java index 050e1bfa320..2e524effc8f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsFormat.java @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.backward_codecs.lucene84; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsReader; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; 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.PostingsFormat; 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.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -94,7 +92,7 @@ import org.apache.lucene.util.packed.PackedInts; *

Term Dictionary *

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 BlockTreeTermsWriter} for more details on the + * .doc, .pos, and .pay files. See {@code BlockTreeTermsWriter} for more details on the * format. *

NOTE: The term dictionary can plug into different postings implementations: the postings * writer/reader are actually responsible for encoding and decoding the PostingsHeader and @@ -149,7 +147,7 @@ import org.apache.lucene.util.packed.PackedInts; *

*
Term Index *

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. *

* * @@ -331,7 +329,7 @@ import org.apache.lucene.util.packed.PackedInts; * * @lucene.experimental */ -public final class Lucene84PostingsFormat extends PostingsFormat { +public class Lucene84PostingsFormat extends PostingsFormat { /** * Filename extension for document number, frequencies, and skip data. See chapter: 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)); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/gen_ForUtil.py b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/gen_ForUtil.py similarity index 100% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/gen_ForUtil.py rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/gen_ForUtil.py diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java index b8659f74d87..817643140fe 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene86/Lucene86Codec.java @@ -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.Lucene50StoredFieldsFormat; 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.CompoundFormat; 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.lucene50.Lucene50TermVectorsFormat; 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.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; @@ -106,7 +106,7 @@ public class Lucene86Codec extends Codec { } @Override - public final PostingsFormat postingsFormat() { + public PostingsFormat postingsFormat() { return postingsFormat; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java index 52bc76c899d..373a7d384e5 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java @@ -21,6 +21,7 @@ import java.util.Objects; import org.apache.lucene.backward_codecs.lucene50.Lucene50CompoundFormat; import org.apache.lucene.backward_codecs.lucene50.Lucene50LiveDocsFormat; 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.CompoundFormat; 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.lucene80.Lucene80DocValuesFormat; 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.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; @@ -118,7 +118,7 @@ public class Lucene87Codec extends Codec { } @Override - public final PostingsFormat postingsFormat() { + public PostingsFormat postingsFormat() { return postingsFormat; } diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index bafc90e9f76..620f7c81a9d 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -14,3 +14,4 @@ # limitations under the License. org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat +org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java index 6467bc7c36e..214085c86a1 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/Lucene87/Lucene87RWCodec.java @@ -17,14 +17,31 @@ package org.apache.lucene.backward_codecs.Lucene87; 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.codecs.CompoundFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; /** RW impersonation of {@link 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 public final CompoundFormat compoundFormat() { return new Lucene50RWCompoundFormat(); } + + @Override + public PostingsFormat postingsFormat() { + return postingsFormat; + } } diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriter.java new file mode 100644 index 00000000000..43878fce198 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriter.java @@ -0,0 +1,1180 @@ +/* + * 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 java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.Fields; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.IntsRefBuilder; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.compress.LZ4; +import org.apache.lucene.util.compress.LowercaseAsciiCompression; +import org.apache.lucene.util.fst.ByteSequenceOutputs; +import org.apache.lucene.util.fst.BytesRefFSTEnum; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.FSTCompiler; +import org.apache.lucene.util.fst.Util; + +/* + TODO: + + - Currently there is a one-to-one mapping of indexed + term to term block, but we could decouple the two, ie, + put more terms into the index than there are blocks. + The index would take up more RAM but then it'd be able + to avoid seeking more often and could make PK/FuzzyQ + faster if the additional indexed terms could store + the offset into the terms block. + + - The blocks are not written in true depth-first + order, meaning if you just next() the file pointer will + sometimes jump backwards. For example, block foo* will + be written before block f* because it finished before. + This could possibly hurt performance if the terms dict is + not hot, since OSs anticipate sequential file access. We + could fix the writer to re-order the blocks as a 2nd + pass. + + - Each block encodes the term suffixes packed + sequentially using a separate vInt per term, which is + 1) wasteful and 2) slow (must linear scan to find a + particular suffix). We should instead 1) make + random-access array so we can directly access the Nth + suffix, and 2) bulk-encode this array using bulk int[] + codecs; then at search time we can binary search when + we seek a particular term. +*/ + +/** + * Block-based terms index and dictionary writer. + * + *

Writes terms dict and index, block-encoding (column stride) each term's metadata for each set + * of terms between two index terms. + * + *

Files: + * + *

+ * + *

+ * + *

Term Dictionary

+ * + *

The .tim file contains the list of terms in each field along with per-term statistics (such as + * docfreq) and per-term metadata (typically pointers to the postings list for that term in the + * inverted index). + * + *

The .tim is arranged in blocks: with blocks containing a variable number of entries (by + * default 25-48), where each entry is either a term or a reference to a sub-block. + * + *

NOTE: The term dictionary can plug into different postings implementations: the postings + * writer/reader are actually responsible for encoding and decoding the Postings Metadata and Term + * Metadata sections. + * + *

    + *
  • TermsDict (.tim) --> Header, PostingsHeader, NodeBlockNumBlocks, + * FieldSummary, DirOffset, Footer + *
  • NodeBlock --> (OuterNode | InnerNode) + *
  • OuterNode --> EntryCount, SuffixLength, ByteSuffixLength, StatsLength, < + * TermStats >EntryCount, MetaLength, + * <TermMetadata>EntryCount + *
  • InnerNode --> EntryCount, SuffixLength[,Sub?], ByteSuffixLength, StatsLength, + * < TermStats ? >EntryCount, MetaLength, <TermMetadata ? + * >EntryCount + *
  • TermStats --> DocFreq, TotalTermFreq + *
  • FieldSummary --> NumFields, <FieldNumber, NumTerms, RootCodeLength, + * ByteRootCodeLength, SumTotalTermFreq?, SumDocFreq, DocCount, LongsSize, MinTerm, + * MaxTerm>NumFields + *
  • Header --> {@link CodecUtil#writeHeader CodecHeader} + *
  • DirOffset --> {@link DataOutput#writeLong Uint64} + *
  • MinTerm,MaxTerm --> {@link DataOutput#writeVInt VInt} length followed by the byte[] + *
  • EntryCount,SuffixLength,StatsLength,DocFreq,MetaLength,NumFields, + * FieldNumber,RootCodeLength,DocCount,LongsSize --> {@link DataOutput#writeVInt VInt} + *
  • TotalTermFreq,NumTerms,SumTotalTermFreq,SumDocFreq --> {@link DataOutput#writeVLong + * VLong} + *
  • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
+ * + *

Notes: + * + *

    + *
  • Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information for + * the BlockTree implementation. + *
  • DirOffset is a pointer to the FieldSummary section. + *
  • DocFreq is the count of documents which contain the term. + *
  • TotalTermFreq is the total number of occurrences of the term. This is encoded as the + * difference between the total number of occurrences and the DocFreq. + *
  • FieldNumber is the fields number from {@link FieldInfos}. (.fnm) + *
  • NumTerms is the number of unique terms for the field. + *
  • RootCode points to the root block for the field. + *
  • SumDocFreq is the total number of postings, the number of term-document pairs across the + * entire field. + *
  • DocCount is the number of documents that have at least one posting for this field. + *
  • LongsSize records how many long values the postings writer/reader record per term (e.g., to + * hold freq/prox/doc file offsets). + *
  • MinTerm, MaxTerm are the lowest and highest term in this field. + *
  • PostingsHeader and TermMetadata are plugged into by the specific postings implementation: + * these contain arbitrary per-file data (such as parameters or versioning information) and + * per-term data (such as pointers to inverted files). + *
  • For inner nodes of the tree, every entry will steal one bit to mark whether it points to + * child nodes(sub-block). If so, the corresponding TermStats and TermMetaData are omitted + *
+ * + * + * + *

Term Index

+ * + *

The .tip file contains an index into the term dictionary, so that it can be accessed randomly. + * The index is also used to determine when a given term cannot exist on disk (in the .tim file), + * saving a disk seek. + * + *

    + *
  • TermsIndex (.tip) --> Header, FSTIndexNumFields + * <IndexStartFP>NumFields, DirOffset, Footer + *
  • Header --> {@link CodecUtil#writeHeader CodecHeader} + *
  • DirOffset --> {@link DataOutput#writeLong Uint64} + *
  • IndexStartFP --> {@link DataOutput#writeVLong VLong} + * + *
  • FSTIndex --> {@link FST FST<byte[]>} + *
  • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
+ * + *

Notes: + * + *

    + *
  • The .tip file contains a separate FST for each field. The FST maps a term prefix to the + * on-disk block that holds all terms starting with that prefix. Each field's IndexStartFP + * points to its FST. + *
  • DirOffset is a pointer to the start of the IndexStartFPs for all fields + *
  • It's possible that an on-disk block would contain too many terms (more than the allowed + * maximum (default: 48)). When this happens, the block is sub-divided into new blocks (called + * "floor blocks"), and then the output in the FST for the block's prefix encodes the leading + * byte of each sub-block, and its file pointer. + *
+ * + * @see Lucene40BlockTreeTermsReader + * @lucene.experimental + */ +public final class Lucene40BlockTreeTermsWriter extends FieldsConsumer { + + /** + * Suggested default value for the {@code minItemsInBlock} parameter to {@link + * #Lucene40BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. + */ + public static final int DEFAULT_MIN_BLOCK_SIZE = 25; + + /** + * Suggested default value for the {@code maxItemsInBlock} parameter to {@link + * #Lucene40BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. + */ + public static final int DEFAULT_MAX_BLOCK_SIZE = 48; + + // public static boolean DEBUG = false; + // public static boolean DEBUG2 = false; + + // private final static boolean SAVE_DOT_FILES = false; + + private final IndexOutput metaOut; + private final IndexOutput termsOut; + private final IndexOutput indexOut; + final int maxDoc; + final int minItemsInBlock; + final int maxItemsInBlock; + + final PostingsWriterBase postingsWriter; + final FieldInfos fieldInfos; + + private final List fields = new ArrayList<>(); + + /** + * Create a new writer. The number of items (terms or sub-blocks) per block will aim to be between + * minItemsPerBlock and maxItemsPerBlock, though in some cases the blocks may be smaller than the + * min. + */ + public Lucene40BlockTreeTermsWriter( + SegmentWriteState state, + PostingsWriterBase postingsWriter, + int minItemsInBlock, + int maxItemsInBlock) + throws IOException { + validateSettings(minItemsInBlock, maxItemsInBlock); + + this.minItemsInBlock = minItemsInBlock; + this.maxItemsInBlock = maxItemsInBlock; + + this.maxDoc = state.segmentInfo.maxDoc(); + this.fieldInfos = state.fieldInfos; + this.postingsWriter = postingsWriter; + + final String termsName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene40BlockTreeTermsReader.TERMS_EXTENSION); + termsOut = state.directory.createOutput(termsName, state.context); + boolean success = false; + IndexOutput metaOut = null, indexOut = null; + try { + CodecUtil.writeIndexHeader( + termsOut, + Lucene40BlockTreeTermsReader.TERMS_CODEC_NAME, + Lucene40BlockTreeTermsReader.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + + final String indexName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene40BlockTreeTermsReader.TERMS_INDEX_EXTENSION); + indexOut = state.directory.createOutput(indexName, state.context); + CodecUtil.writeIndexHeader( + indexOut, + Lucene40BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, + Lucene40BlockTreeTermsReader.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + // segment = state.segmentInfo.name; + + final String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene40BlockTreeTermsReader.TERMS_META_EXTENSION); + metaOut = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader( + metaOut, + Lucene40BlockTreeTermsReader.TERMS_META_CODEC_NAME, + Lucene40BlockTreeTermsReader.VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + + postingsWriter.init(metaOut, state); // have consumer write its format/header + + this.metaOut = metaOut; + this.indexOut = indexOut; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(metaOut, termsOut, indexOut); + } + } + } + + /** Throws {@code IllegalArgumentException} if any of these settings is invalid. */ + public static void validateSettings(int minItemsInBlock, int maxItemsInBlock) { + if (minItemsInBlock <= 1) { + throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock); + } + if (minItemsInBlock > maxItemsInBlock) { + throw new IllegalArgumentException( + "maxItemsInBlock must be >= minItemsInBlock; got maxItemsInBlock=" + + maxItemsInBlock + + " minItemsInBlock=" + + minItemsInBlock); + } + if (2 * (minItemsInBlock - 1) > maxItemsInBlock) { + throw new IllegalArgumentException( + "maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + + maxItemsInBlock + + " minItemsInBlock=" + + minItemsInBlock); + } + } + + @Override + public void write(Fields fields, NormsProducer norms) throws IOException { + // if (DEBUG) System.out.println("\nBTTW.write seg=" + segment); + + String lastField = null; + for (String field : fields) { + assert lastField == null || lastField.compareTo(field) < 0; + lastField = field; + + // if (DEBUG) System.out.println("\nBTTW.write seg=" + segment + " field=" + field); + Terms terms = fields.terms(field); + if (terms == null) { + continue; + } + + TermsEnum termsEnum = terms.iterator(); + TermsWriter termsWriter = new TermsWriter(fieldInfos.fieldInfo(field)); + while (true) { + BytesRef term = termsEnum.next(); + // if (DEBUG) System.out.println("BTTW: next term " + term); + + if (term == null) { + break; + } + + // if (DEBUG) System.out.println("write field=" + fieldInfo.name + " term=" + + // brToString(term)); + termsWriter.write(term, termsEnum, norms); + } + + termsWriter.finish(); + + // if (DEBUG) System.out.println("\nBTTW.write done seg=" + segment + " field=" + field); + } + } + + static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) { + assert fp < (1L << 62); + return (fp << 2) + | (hasTerms ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + | (isFloor ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); + } + + private static class PendingEntry { + public final boolean isTerm; + + protected PendingEntry(boolean isTerm) { + this.isTerm = isTerm; + } + } + + private static final class PendingTerm extends PendingEntry { + public final byte[] termBytes; + // stats + metadata + public final BlockTermState state; + + public PendingTerm(BytesRef term, BlockTermState state) { + super(true); + this.termBytes = new byte[term.length]; + System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length); + this.state = state; + } + + @Override + public String toString() { + return "TERM: " + brToString(termBytes); + } + } + + // for debugging + @SuppressWarnings("unused") + static 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(); + } + } + } + + // for debugging + @SuppressWarnings("unused") + static String brToString(byte[] b) { + return brToString(new BytesRef(b)); + } + + private static final class PendingBlock extends PendingEntry { + public final BytesRef prefix; + public final long fp; + public FST index; + public List> subIndices; + public final boolean hasTerms; + public final boolean isFloor; + public final int floorLeadByte; + + public PendingBlock( + BytesRef prefix, + long fp, + boolean hasTerms, + boolean isFloor, + int floorLeadByte, + List> subIndices) { + super(false); + this.prefix = prefix; + this.fp = fp; + this.hasTerms = hasTerms; + this.isFloor = isFloor; + this.floorLeadByte = floorLeadByte; + this.subIndices = subIndices; + } + + @Override + public String toString() { + return "BLOCK: prefix=" + brToString(prefix); + } + + public void compileIndex( + List blocks, + ByteBuffersDataOutput scratchBytes, + IntsRefBuilder scratchIntsRef) + throws IOException { + + assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1) + : "isFloor=" + isFloor + " blocks=" + blocks; + assert this == blocks.get(0); + + assert scratchBytes.size() == 0; + + // TODO: try writing the leading vLong in MSB order + // (opposite of what Lucene does today), for better + // outputs sharing in the FST + scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor)); + if (isFloor) { + scratchBytes.writeVInt(blocks.size() - 1); + for (int i = 1; i < blocks.size(); i++) { + PendingBlock sub = blocks.get(i); + assert sub.floorLeadByte != -1; + // if (DEBUG) { + // System.out.println(" write floorLeadByte=" + + // Integer.toHexString(sub.floorLeadByte&0xff)); + // } + scratchBytes.writeByte((byte) sub.floorLeadByte); + assert sub.fp > fp; + scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0)); + } + } + + final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton(); + final FSTCompiler fstCompiler = + new FSTCompiler.Builder<>(FST.INPUT_TYPE.BYTE1, outputs) + .shouldShareNonSingletonNodes(false) + .build(); + // if (DEBUG) { + // System.out.println(" compile index for prefix=" + prefix); + // } + // indexBuilder.DEBUG = false; + final byte[] bytes = scratchBytes.toArrayCopy(); + assert bytes.length > 0; + fstCompiler.add(Util.toIntsRef(prefix, scratchIntsRef), new BytesRef(bytes, 0, bytes.length)); + scratchBytes.reset(); + + // Copy over index for all sub-blocks + for (PendingBlock block : blocks) { + if (block.subIndices != null) { + for (FST subIndex : block.subIndices) { + append(fstCompiler, subIndex, scratchIntsRef); + } + block.subIndices = null; + } + } + + index = fstCompiler.compile(); + + assert subIndices == null; + + /* + Writer w = new OutputStreamWriter(new FileOutputStream("out.dot")); + Util.toDot(index, w, false, false); + System.out.println("SAVED to out.dot"); + w.close(); + */ + } + + // TODO: maybe we could add bulk-add method to + // Builder? Takes FST and unions it w/ current + // FST. + private void append( + FSTCompiler fstCompiler, FST subIndex, IntsRefBuilder scratchIntsRef) + throws IOException { + final BytesRefFSTEnum subIndexEnum = new BytesRefFSTEnum<>(subIndex); + BytesRefFSTEnum.InputOutput indexEnt; + while ((indexEnt = subIndexEnum.next()) != null) { + // if (DEBUG) { + // System.out.println(" add sub=" + indexEnt.input + " " + indexEnt.input + " output=" + // + indexEnt.output); + // } + fstCompiler.add(Util.toIntsRef(indexEnt.input, scratchIntsRef), indexEnt.output); + } + } + } + + private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance(); + private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder(); + + static final BytesRef EMPTY_BYTES_REF = new BytesRef(); + + private static class StatsWriter { + + private final DataOutput out; + private final boolean hasFreqs; + private int singletonCount; + + StatsWriter(DataOutput out, boolean hasFreqs) { + this.out = out; + this.hasFreqs = hasFreqs; + } + + void add(int df, long ttf) throws IOException { + // Singletons (DF==1, TTF==1) are run-length encoded + if (df == 1 && (hasFreqs == false || ttf == 1)) { + singletonCount++; + } else { + finish(); + out.writeVInt(df << 1); + if (hasFreqs) { + out.writeVLong(ttf - df); + } + } + } + + void finish() throws IOException { + if (singletonCount > 0) { + out.writeVInt(((singletonCount - 1) << 1) | 1); + singletonCount = 0; + } + } + } + + class TermsWriter { + private final FieldInfo fieldInfo; + private long numTerms; + final FixedBitSet docsSeen; + long sumTotalTermFreq; + long sumDocFreq; + + // Records index into pending where the current prefix at that + // length "started"; for example, if current term starts with 't', + // startsByPrefix[0] is the index into pending for the first + // term/sub-block starting with 't'. We use this to figure out when + // to write a new block: + private final BytesRefBuilder lastTerm = new BytesRefBuilder(); + private int[] prefixStarts = new int[8]; + + // Pending stack of terms and blocks. As terms arrive (in sorted order) + // we append to this stack, and once the top of the stack has enough + // terms starting with a common prefix, we write a new block with + // those terms and replace those terms in the stack with a new block: + private final List pending = new ArrayList<>(); + + // Reused in writeBlocks: + private final List newBlocks = new ArrayList<>(); + + private PendingTerm firstPendingTerm; + private PendingTerm lastPendingTerm; + + /** Writes the top count entries in pending, using prevTerm to compute the prefix. */ + void writeBlocks(int prefixLength, int count) throws IOException { + + assert count > 0; + + // if (DEBUG2) { + // BytesRef br = new BytesRef(lastTerm.bytes()); + // br.length = prefixLength; + // System.out.println("writeBlocks: seg=" + segment + " prefix=" + brToString(br) + " count=" + // + count); + // } + + // Root block better write all remaining pending entries: + assert prefixLength > 0 || count == pending.size(); + + int lastSuffixLeadLabel = -1; + + // True if we saw at least one term in this block (we record if a block + // only points to sub-blocks in the terms index so we can avoid seeking + // to it when we are looking for a term): + boolean hasTerms = false; + boolean hasSubBlocks = false; + + int start = pending.size() - count; + int end = pending.size(); + int nextBlockStart = start; + int nextFloorLeadLabel = -1; + + for (int i = start; i < end; i++) { + + PendingEntry ent = pending.get(i); + + int suffixLeadLabel; + + if (ent.isTerm) { + PendingTerm term = (PendingTerm) ent; + if (term.termBytes.length == prefixLength) { + // Suffix is 0, i.e. prefix 'foo' and term is + // 'foo' so the term has empty string suffix + // in this block + assert lastSuffixLeadLabel == -1 + : "i=" + i + " lastSuffixLeadLabel=" + lastSuffixLeadLabel; + suffixLeadLabel = -1; + } else { + suffixLeadLabel = term.termBytes[prefixLength] & 0xff; + } + } else { + PendingBlock block = (PendingBlock) ent; + assert block.prefix.length > prefixLength; + suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff; + } + // if (DEBUG) System.out.println(" i=" + i + " ent=" + ent + " suffixLeadLabel=" + + // suffixLeadLabel); + + if (suffixLeadLabel != lastSuffixLeadLabel) { + int itemsInBlock = i - nextBlockStart; + if (itemsInBlock >= minItemsInBlock && end - nextBlockStart > maxItemsInBlock) { + // The count is too large for one block, so we must break it into "floor" blocks, where + // we record + // the leading label of the suffix of the first term in each floor block, so at search + // time we can + // jump to the right floor block. We just use a naive greedy segmenter here: make a new + // floor + // block as soon as we have at least minItemsInBlock. This is not always best: it often + // produces + // a too-small block as the final block: + boolean isFloor = itemsInBlock < count; + newBlocks.add( + writeBlock( + prefixLength, + isFloor, + nextFloorLeadLabel, + nextBlockStart, + i, + hasTerms, + hasSubBlocks)); + + hasTerms = false; + hasSubBlocks = false; + nextFloorLeadLabel = suffixLeadLabel; + nextBlockStart = i; + } + + lastSuffixLeadLabel = suffixLeadLabel; + } + + if (ent.isTerm) { + hasTerms = true; + } else { + hasSubBlocks = true; + } + } + + // Write last block, if any: + if (nextBlockStart < end) { + int itemsInBlock = end - nextBlockStart; + boolean isFloor = itemsInBlock < count; + newBlocks.add( + writeBlock( + prefixLength, + isFloor, + nextFloorLeadLabel, + nextBlockStart, + end, + hasTerms, + hasSubBlocks)); + } + + assert newBlocks.isEmpty() == false; + + PendingBlock firstBlock = newBlocks.get(0); + + assert firstBlock.isFloor || newBlocks.size() == 1; + + firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef); + + // Remove slice from the top of the pending stack, that we just wrote: + pending.subList(pending.size() - count, pending.size()).clear(); + + // Append new block + pending.add(firstBlock); + + newBlocks.clear(); + } + + private boolean allEqual(byte[] b, int startOffset, int endOffset, byte value) { + Objects.checkFromToIndex(startOffset, endOffset, b.length); + for (int i = startOffset; i < endOffset; ++i) { + if (b[i] != value) { + return false; + } + } + return true; + } + + /** + * Writes the specified slice (start is inclusive, end is exclusive) from pending stack as a new + * block. If isFloor is true, there were too many (more than maxItemsInBlock) entries sharing + * the same prefix, and so we broke it into multiple floor blocks where we record the starting + * label of the suffix of each floor block. + */ + private PendingBlock writeBlock( + int prefixLength, + boolean isFloor, + int floorLeadLabel, + int start, + int end, + boolean hasTerms, + boolean hasSubBlocks) + throws IOException { + + assert end > start; + + long startFP = termsOut.getFilePointer(); + + boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1; + + final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0)); + System.arraycopy(lastTerm.get().bytes, 0, prefix.bytes, 0, prefixLength); + prefix.length = prefixLength; + + // if (DEBUG2) System.out.println(" writeBlock field=" + fieldInfo.name + " prefix=" + + // brToString(prefix) + " fp=" + startFP + " isFloor=" + isFloor + " isLastInFloor=" + (end == + // pending.size()) + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + + // " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks); + + // Write block header: + int numEntries = end - start; + int code = numEntries << 1; + if (end == pending.size()) { + // Last block: + code |= 1; + } + termsOut.writeVInt(code); + + /* + if (DEBUG) { + System.out.println(" writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + brToString(prefix) + " entCount=" + (end-start+1) + " startFP=" + startFP + (isFloor ? (" floorLeadLabel=" + Integer.toHexString(floorLeadLabel)) : "")); + } + */ + + // 1st pass: pack term suffix bytes into byte[] blob + // TODO: cutover to bulk int codec... simple64? + + // We optimize the leaf block case (block has only terms), writing a more + // compact format in this case: + boolean isLeafBlock = hasSubBlocks == false; + + // System.out.println(" isLeaf=" + isLeafBlock); + + final List> subIndices; + + boolean absolute = true; + + if (isLeafBlock) { + // Block contains only ordinary terms: + subIndices = null; + StatsWriter statsWriter = + new StatsWriter(this.statsWriter, fieldInfo.getIndexOptions() != IndexOptions.DOCS); + for (int i = start; i < end; i++) { + PendingEntry ent = pending.get(i); + assert ent.isTerm : "i=" + i; + + PendingTerm term = (PendingTerm) ent; + + assert StringHelper.startsWith(term.termBytes, prefix) : term + " prefix=" + prefix; + BlockTermState state = term.state; + final int suffix = term.termBytes.length - prefixLength; + // if (DEBUG2) { + // BytesRef suffixBytes = new BytesRef(suffix); + // System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix); + // suffixBytes.length = suffix; + // System.out.println(" write term suffix=" + brToString(suffixBytes)); + // } + + // For leaf block we write suffix straight + suffixLengthsWriter.writeVInt(suffix); + suffixWriter.append(term.termBytes, prefixLength, suffix); + assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel; + + // Write term stats, to separate byte[] blob: + statsWriter.add(state.docFreq, state.totalTermFreq); + + // Write term meta data + postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute); + absolute = false; + } + statsWriter.finish(); + } else { + // Block has at least one prefix term or a sub block: + subIndices = new ArrayList<>(); + StatsWriter statsWriter = + new StatsWriter(this.statsWriter, fieldInfo.getIndexOptions() != IndexOptions.DOCS); + for (int i = start; i < end; i++) { + PendingEntry ent = pending.get(i); + if (ent.isTerm) { + PendingTerm term = (PendingTerm) ent; + + assert StringHelper.startsWith(term.termBytes, prefix) : term + " prefix=" + prefix; + BlockTermState state = term.state; + final int suffix = term.termBytes.length - prefixLength; + // if (DEBUG2) { + // BytesRef suffixBytes = new BytesRef(suffix); + // System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix); + // suffixBytes.length = suffix; + // System.out.println(" write term suffix=" + brToString(suffixBytes)); + // } + + // For non-leaf block we borrow 1 bit to record + // if entry is term or sub-block, and 1 bit to record if + // it's a prefix term. Terms cannot be larger than ~32 KB + // so we won't run out of bits: + + suffixLengthsWriter.writeVInt(suffix << 1); + suffixWriter.append(term.termBytes, prefixLength, suffix); + + // Write term stats, to separate byte[] blob: + statsWriter.add(state.docFreq, state.totalTermFreq); + + // TODO: now that terms dict "sees" these longs, + // we can explore better column-stride encodings + // to encode all long[0]s for this block at + // once, all long[1]s, etc., e.g. using + // Simple64. Alternatively, we could interleave + // stats + meta ... no reason to have them + // separate anymore: + + // Write term meta data + postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute); + absolute = false; + } else { + PendingBlock block = (PendingBlock) ent; + assert StringHelper.startsWith(block.prefix, prefix); + final int suffix = block.prefix.length - prefixLength; + assert StringHelper.startsWith(block.prefix, prefix); + + assert suffix > 0; + + // For non-leaf block we borrow 1 bit to record + // if entry is term or sub-block:f + suffixLengthsWriter.writeVInt((suffix << 1) | 1); + suffixWriter.append(block.prefix.bytes, prefixLength, suffix); + + // if (DEBUG2) { + // BytesRef suffixBytes = new BytesRef(suffix); + // System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix); + // suffixBytes.length = suffix; + // System.out.println(" write sub-block suffix=" + brToString(suffixBytes) + " + // subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor); + // } + + assert floorLeadLabel == -1 + || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel + : "floorLeadLabel=" + + floorLeadLabel + + " suffixLead=" + + (block.prefix.bytes[prefixLength] & 0xff); + assert block.fp < startFP; + + suffixLengthsWriter.writeVLong(startFP - block.fp); + subIndices.add(block.index); + } + } + statsWriter.finish(); + + assert subIndices.size() != 0; + } + + // Write suffixes byte[] blob to terms dict output, either uncompressed, compressed with LZ4 + // or with LowercaseAsciiCompression. + CompressionAlgorithm compressionAlg = CompressionAlgorithm.NO_COMPRESSION; + // If there are 2 suffix bytes or less per term, then we don't bother compressing as suffix + // are unlikely what + // makes the terms dictionary large, and it also tends to be frequently the case for dense IDs + // like + // auto-increment IDs, so not compressing in that case helps not hurt ID lookups by too much. + // We also only start compressing when the prefix length is greater than 2 since blocks whose + // prefix length is + // 1 or 2 always all get visited when running a fuzzy query whose max number of edits is 2. + if (suffixWriter.length() > 2L * numEntries && prefixLength > 2) { + // LZ4 inserts references whenever it sees duplicate strings of 4 chars or more, so only try + // it out if the + // average suffix length is greater than 6. + if (suffixWriter.length() > 6L * numEntries) { + LZ4.compress( + suffixWriter.bytes(), 0, suffixWriter.length(), spareWriter, compressionHashTable); + if (spareWriter.size() < suffixWriter.length() - (suffixWriter.length() >>> 2)) { + // LZ4 saved more than 25%, go for it + compressionAlg = CompressionAlgorithm.LZ4; + } + } + if (compressionAlg == CompressionAlgorithm.NO_COMPRESSION) { + spareWriter.reset(); + if (spareBytes.length < suffixWriter.length()) { + spareBytes = new byte[ArrayUtil.oversize(suffixWriter.length(), 1)]; + } + if (LowercaseAsciiCompression.compress( + suffixWriter.bytes(), suffixWriter.length(), spareBytes, spareWriter)) { + compressionAlg = CompressionAlgorithm.LOWERCASE_ASCII; + } + } + } + long token = ((long) suffixWriter.length()) << 3; + if (isLeafBlock) { + token |= 0x04; + } + token |= compressionAlg.code; + termsOut.writeVLong(token); + if (compressionAlg == CompressionAlgorithm.NO_COMPRESSION) { + termsOut.writeBytes(suffixWriter.bytes(), suffixWriter.length()); + } else { + spareWriter.copyTo(termsOut); + } + suffixWriter.setLength(0); + spareWriter.reset(); + + // Write suffix lengths + final int numSuffixBytes = Math.toIntExact(suffixLengthsWriter.size()); + spareBytes = ArrayUtil.grow(spareBytes, numSuffixBytes); + suffixLengthsWriter.copyTo(new ByteArrayDataOutput(spareBytes)); + suffixLengthsWriter.reset(); + if (allEqual(spareBytes, 1, numSuffixBytes, spareBytes[0])) { + // Structured fields like IDs often have most values of the same length + termsOut.writeVInt((numSuffixBytes << 1) | 1); + termsOut.writeByte(spareBytes[0]); + } else { + termsOut.writeVInt(numSuffixBytes << 1); + termsOut.writeBytes(spareBytes, numSuffixBytes); + } + + // Stats + final int numStatsBytes = Math.toIntExact(statsWriter.size()); + termsOut.writeVInt(numStatsBytes); + statsWriter.copyTo(termsOut); + statsWriter.reset(); + + // Write term meta data byte[] blob + termsOut.writeVInt((int) metaWriter.size()); + metaWriter.copyTo(termsOut); + metaWriter.reset(); + + // if (DEBUG) { + // System.out.println(" fpEnd=" + out.getFilePointer()); + // } + + if (hasFloorLeadLabel) { + // We already allocated to length+1 above: + prefix.bytes[prefix.length++] = (byte) floorLeadLabel; + } + + return new PendingBlock(prefix, startFP, hasTerms, isFloor, floorLeadLabel, subIndices); + } + + TermsWriter(FieldInfo fieldInfo) { + this.fieldInfo = fieldInfo; + assert fieldInfo.getIndexOptions() != IndexOptions.NONE; + docsSeen = new FixedBitSet(maxDoc); + postingsWriter.setField(fieldInfo); + } + + /** Writes one term's worth of postings. */ + public void write(BytesRef text, TermsEnum termsEnum, NormsProducer norms) throws IOException { + /* + if (DEBUG) { + int[] tmp = new int[lastTerm.length]; + System.arraycopy(prefixStarts, 0, tmp, 0, tmp.length); + System.out.println("BTTW: write term=" + brToString(text) + " prefixStarts=" + Arrays.toString(tmp) + " pending.size()=" + pending.size()); + } + */ + + BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen, norms); + if (state != null) { + + assert state.docFreq != 0; + assert fieldInfo.getIndexOptions() == IndexOptions.DOCS + || state.totalTermFreq >= state.docFreq + : "postingsWriter=" + postingsWriter; + pushTerm(text); + + PendingTerm term = new PendingTerm(text, state); + pending.add(term); + // if (DEBUG) System.out.println(" add pending term = " + text + " pending.size()=" + + // pending.size()); + + sumDocFreq += state.docFreq; + sumTotalTermFreq += state.totalTermFreq; + numTerms++; + if (firstPendingTerm == null) { + firstPendingTerm = term; + } + lastPendingTerm = term; + } + } + + /** Pushes the new term to the top of the stack, and writes new blocks. */ + private void pushTerm(BytesRef text) throws IOException { + // Find common prefix between last term and current term: + int prefixLength = + Arrays.mismatch( + lastTerm.bytes(), + 0, + lastTerm.length(), + text.bytes, + text.offset, + text.offset + text.length); + if (prefixLength == -1) { // Only happens for the first term, if it is empty + assert lastTerm.length() == 0; + prefixLength = 0; + } + + // if (DEBUG) System.out.println(" shared=" + pos + " lastTerm.length=" + lastTerm.length); + + // Close the "abandoned" suffix now: + for (int i = lastTerm.length() - 1; i >= prefixLength; i--) { + + // How many items on top of the stack share the current suffix + // we are closing: + int prefixTopSize = pending.size() - prefixStarts[i]; + if (prefixTopSize >= minItemsInBlock) { + // if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " + // minItemsInBlock=" + minItemsInBlock); + writeBlocks(i + 1, prefixTopSize); + prefixStarts[i] -= prefixTopSize - 1; + } + } + + if (prefixStarts.length < text.length) { + prefixStarts = ArrayUtil.grow(prefixStarts, text.length); + } + + // Init new tail: + for (int i = prefixLength; i < text.length; i++) { + prefixStarts[i] = pending.size(); + } + + lastTerm.copyBytes(text); + } + + // Finishes all terms in this field + public void finish() throws IOException { + if (numTerms > 0) { + // if (DEBUG) System.out.println("BTTW: finish prefixStarts=" + + // Arrays.toString(prefixStarts)); + + // Add empty term to force closing of all final blocks: + pushTerm(new BytesRef()); + + // TODO: if pending.size() is already 1 with a non-zero prefix length + // we can save writing a "degenerate" root block, but we have to + // fix all the places that assume the root block's prefix is the empty string: + pushTerm(new BytesRef()); + writeBlocks(0, pending.size()); + + // We better have one final "root" block: + assert pending.size() == 1 && !pending.get(0).isTerm + : "pending.size()=" + pending.size() + " pending=" + pending; + final PendingBlock root = (PendingBlock) pending.get(0); + assert root.prefix.length == 0; + final BytesRef rootCode = root.index.getEmptyOutput(); + assert rootCode != null; + + ByteBuffersDataOutput metaOut = new ByteBuffersDataOutput(); + fields.add(metaOut); + + metaOut.writeVInt(fieldInfo.number); + metaOut.writeVLong(numTerms); + metaOut.writeVInt(rootCode.length); + metaOut.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length); + assert fieldInfo.getIndexOptions() != IndexOptions.NONE; + if (fieldInfo.getIndexOptions() != IndexOptions.DOCS) { + metaOut.writeVLong(sumTotalTermFreq); + } + metaOut.writeVLong(sumDocFreq); + metaOut.writeVInt(docsSeen.cardinality()); + writeBytesRef(metaOut, new BytesRef(firstPendingTerm.termBytes)); + writeBytesRef(metaOut, new BytesRef(lastPendingTerm.termBytes)); + metaOut.writeVLong(indexOut.getFilePointer()); + // Write FST to index + root.index.save(metaOut, indexOut); + // System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name); + + /* + if (DEBUG) { + final String dotFileName = segment + "_" + fieldInfo.name + ".dot"; + Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName)); + Util.toDot(root.index, w, false, false); + System.out.println("SAVED to " + dotFileName); + w.close(); + } + */ + + } else { + assert sumTotalTermFreq == 0 + || fieldInfo.getIndexOptions() == IndexOptions.DOCS && sumTotalTermFreq == -1; + assert sumDocFreq == 0; + assert docsSeen.cardinality() == 0; + } + } + + private final ByteBuffersDataOutput suffixLengthsWriter = + ByteBuffersDataOutput.newResettableInstance(); + private final BytesRefBuilder suffixWriter = new BytesRefBuilder(); + private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance(); + private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance(); + private final ByteBuffersDataOutput spareWriter = ByteBuffersDataOutput.newResettableInstance(); + private byte[] spareBytes = BytesRef.EMPTY_BYTES; + private final LZ4.HighCompressionHashTable compressionHashTable = + new LZ4.HighCompressionHashTable(); + } + + private boolean closed; + + @Override + public void close() throws IOException { + if (closed) { + return; + } + closed = true; + + boolean success = false; + try { + metaOut.writeVInt(fields.size()); + for (ByteBuffersDataOutput fieldMeta : fields) { + fieldMeta.copyTo(metaOut); + } + CodecUtil.writeFooter(indexOut); + metaOut.writeLong(indexOut.getFilePointer()); + CodecUtil.writeFooter(termsOut); + metaOut.writeLong(termsOut.getFilePointer()); + CodecUtil.writeFooter(metaOut); + success = true; + } finally { + if (success) { + IOUtils.close(metaOut, termsOut, indexOut, postingsWriter); + } else { + IOUtils.closeWhileHandlingException(metaOut, termsOut, indexOut, postingsWriter); + } + } + } + + private static void writeBytesRef(DataOutput out, BytesRef bytes) throws IOException { + out.writeVInt(bytes.length); + out.writeBytes(bytes.bytes, bytes.offset, bytes.length); + } +} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java index 5cedf286161..7b5f0b3ebc2 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java @@ -17,9 +17,9 @@ package org.apache.lucene.backward_codecs.lucene50; 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.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -31,11 +31,11 @@ public class Lucene50RWPostingsFormat extends Lucene50PostingsFormat { boolean success = false; try { FieldsConsumer ret = - new BlockTreeTermsWriter( + new Lucene40BlockTreeTermsWriter( state, postingsWriter, - BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, - BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + Lucene40BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene40BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); success = true; return ret; } finally { diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestBlockPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestBlockPostingsFormat.java index 1bb3b0c630f..bb2bffa6a74 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestBlockPostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/TestBlockPostingsFormat.java @@ -21,11 +21,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; 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.codecs.Codec; 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.Field; import org.apache.lucene.index.BasePostingsFormatTestCase; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsWriter.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsWriter.java index 0ddc3a79c44..67ab62feae3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84PostingsWriter.java @@ -14,23 +14,23 @@ * See the License for the specific language governing permissions and * 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.codecs.lucene84.Lucene84PostingsFormat.DOC_CODEC; -import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.MAX_SKIP_LEVELS; -import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.PAY_CODEC; -import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.POS_CODEC; -import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.TERMS_CODEC; -import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene84.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.DOC_CODEC; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.MAX_SKIP_LEVELS; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.PAY_CODEC; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.POS_CODEC; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.VERSION_CURRENT; import java.io.IOException; import java.nio.ByteOrder; +import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.PushPostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexFileNames; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java index 05736d94e00..a5e34c90fdc 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWCodec.java @@ -22,12 +22,23 @@ import org.apache.lucene.backward_codecs.lucene60.Lucene60RWPointsFormat; import org.apache.lucene.backward_codecs.lucene70.Lucene70RWSegmentInfoFormat; import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.PointsFormat; +import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; /** RW impersonation of {@link 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 public PointsFormat pointsFormat() { return new Lucene60RWPointsFormat(); @@ -43,6 +54,11 @@ public class Lucene84RWCodec extends Lucene84Codec { return new Lucene50RWStoredFieldsFormat(); } + @Override + public PostingsFormat postingsFormat() { + return postingsFormat; + } + @Override public final CompoundFormat compoundFormat() { return new Lucene50RWCompoundFormat(); diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWPostingsFormat.java new file mode 100644 index 00000000000..0b88d6e7dcf --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84RWPostingsFormat.java @@ -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); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84SkipWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84SkipWriter.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84SkipWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84SkipWriter.java index f9d8d12b03c..45289a460df 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84SkipWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/Lucene84SkipWriter.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.backward_codecs.lucene84; import java.io.IOException; import java.util.Arrays; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForDeltaUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForDeltaUtil.java new file mode 100644 index 00000000000..3023c236f54 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForDeltaUtil.java @@ -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(); + } +} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForUtil.java new file mode 100644 index 00000000000..5cfc142c70f --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestForUtil.java @@ -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(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestLucene84PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestLucene84PostingsFormat.java similarity index 87% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestLucene84PostingsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestLucene84PostingsFormat.java index 6d45866ef80..e1d8b3e9743 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestLucene84PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestLucene84PostingsFormat.java @@ -14,18 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.backward_codecs.lucene84; 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.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.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.Field; import org.apache.lucene.index.BasePostingsFormatTestCase; @@ -41,7 +41,7 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.TestUtil; public class TestLucene84PostingsFormat extends BasePostingsFormatTestCase { - private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene84PostingsFormat()); + private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene84RWPostingsFormat()); @Override protected Codec getCodec() { @@ -73,22 +73,6 @@ public class TestLucene84PostingsFormat extends BasePostingsFormatTestCase { 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 { // omit norms and omit freqs doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestPForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestPForUtil.java new file mode 100644 index 00000000000..733974caa06 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene84/TestPForUtil.java @@ -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(); + } +} diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java index c1d278f38d2..c8dd36b5dbc 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene86/Lucene86RWCodec.java @@ -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.Lucene50RWStoredFieldsFormat; 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.PostingsFormat; import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; /** RW impersonation of {@link Lucene86Codec}. */ public class Lucene86RWCodec extends Lucene86Codec { 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. */ public Lucene86RWCodec() { @@ -42,6 +53,11 @@ public class Lucene86RWCodec extends Lucene86Codec { return storedFieldsFormat; } + @Override + public PostingsFormat postingsFormat() { + return postingsFormat; + } + @Override public final CompoundFormat compoundFormat() { return new Lucene50RWCompoundFormat(); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java index dd7db61bee0..5ee3451c9e8 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java @@ -22,14 +22,14 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; -/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene84PostingsWriter}. */ +/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene90PostingsWriter}. */ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { private final int minTermBlockSize; @@ -57,7 +57,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { super("BlockTreeOrds"); this.minTermBlockSize = minTermBlockSize; this.maxTermBlockSize = maxTermBlockSize; - BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); + Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); } @Override @@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state); boolean success = false; try { @@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene90PostingsReader(state); boolean success = false; try { FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java index 05a58770f00..ecb6cffa9df 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java @@ -24,8 +24,8 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.NormsProducer; 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.lucene90.blocktree.Lucene90BlockTreeTermsWriter; // javadocs import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; 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 - * adds a method to its TermsEnum implementation to seekExact only if the version is >= the + * This is just like {@link Lucene90BlockTreeTermsWriter}, except it also stores a version per term, + * 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 * block has a high enough version. The term blocks file is .tiv and the terms index extension is * .tipv. @@ -188,7 +188,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer { int minItemsInBlock, int maxItemsInBlock) throws IOException { - BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); + Lucene90BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); maxDoc = state.segmentInfo.maxDoc(); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java index 77b5c9e857b..a13522ba218 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java @@ -25,7 +25,7 @@ import java.util.TreeMap; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.FieldInfo; 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? /** - * 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[]. * *

WARNING: This is exceptionally RAM intensive: it makes no effort to compress the @@ -98,12 +98,12 @@ public final class DirectPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return PostingsFormat.forName("Lucene84").fieldsConsumer(state); + return PostingsFormat.forName("Lucene90").fieldsConsumer(state); } @Override 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) { FieldsProducer loadedPostings; try { diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java index 666662be666..4e0fec1587f 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java @@ -22,8 +22,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -41,7 +41,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state); boolean success = false; try { @@ -57,7 +57,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene90PostingsReader(state); boolean success = false; try { FieldsProducer ret = new FSTTermsReader(state, postingsReader); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java index f88e7dad0a8..458c6c07cec 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java @@ -17,13 +17,13 @@ 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 org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.IntBlockTermState; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.TermState; @@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator; /** * {@link TermState} serializer which encodes each file pointer as a delta relative to a base file - * pointer. It differs from {@link 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. * *

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}. * - *

Simpler variant of {@link Lucene84PostingsWriter#encodeTerm(DataOutput, FieldInfo, + *

Simpler variant of {@link Lucene90PostingsWriter#encodeTerm(DataOutput, FieldInfo, * BlockTermState, boolean)}. */ public void writeTermState( @@ -148,7 +148,7 @@ public class DeltaBaseTermStateSerializer implements Accountable { /** * Reads a {@link BlockTermState} from the provided {@link DataInput}. * - *

Simpler variant of {@link Lucene84PostingsReader#decodeTerm(DataInput, FieldInfo, + *

Simpler variant of {@link Lucene90PostingsReader#decodeTerm(DataInput, FieldInfo, * BlockTermState, boolean)}. * * @param reuse {@link BlockTermState} to reuse; or null to create a new one. diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java index 2b7a5e20505..d0038078f1e 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/IntersectBlockReader.java @@ -38,7 +38,7 @@ import org.apache.lucene.util.automaton.Transition; * automaton. * *

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.WildcardQuery} (-5%) and slightly faster for {@link * org.apache.lucene.search.PrefixQuery} (+5%). diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java index f04eb0b30f4..c3ccd50e1aa 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -110,7 +110,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state); boolean success = false; try { FieldsConsumer termsWriter = @@ -127,7 +127,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene90PostingsReader(state); boolean success = false; try { FieldsProducer termsReader = diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java index 42c82434e0a..f1df01d09b2 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java @@ -28,7 +28,7 @@ * org.apache.lucene.search.PhraseQuery}) *

  • Quite efficient for {@link org.apache.lucene.search.PrefixQuery} *
  • 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} * */ package org.apache.lucene.codecs.uniformsplit; diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene84/MockTermStateFactory.java b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/MockTermStateFactory.java similarity index 75% rename from lucene/codecs/src/test/org/apache/lucene/codecs/lucene84/MockTermStateFactory.java rename to lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/MockTermStateFactory.java index 92b01660024..17ee5ef8bb0 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene84/MockTermStateFactory.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/MockTermStateFactory.java @@ -15,13 +15,13 @@ * 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 { - /** Creates an empty {@link Lucene84PostingsFormat.IntBlockTermState}. */ - public static Lucene84PostingsFormat.IntBlockTermState create() { - return new Lucene84PostingsFormat.IntBlockTermState(); + /** Creates an empty {@link Lucene90PostingsFormat.IntBlockTermState}. */ + public static Lucene90PostingsFormat.IntBlockTermState create() { + return new Lucene90PostingsFormat.IntBlockTermState(); } } diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/TestBlockWriter.java b/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/TestBlockWriter.java index 799be279b8f..b73053f9763 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/TestBlockWriter.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/TestBlockWriter.java @@ -19,7 +19,7 @@ package org.apache.lucene.codecs.uniformsplit; import java.io.IOException; 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.FieldInfo; import org.apache.lucene.index.IndexOptions; diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/sharedterms/TestSTBlockReader.java b/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/sharedterms/TestSTBlockReader.java index 019713217d8..49d15fc1ad4 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/sharedterms/TestSTBlockReader.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/uniformsplit/sharedterms/TestSTBlockReader.java @@ -27,7 +27,7 @@ import java.util.Map; import java.util.Set; import org.apache.lucene.codecs.BlockTermState; 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.BlockLine; import org.apache.lucene.codecs.uniformsplit.FSTDictionary; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java index a98dc3bc5fd..cb1c874b0ea 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java @@ -18,7 +18,7 @@ package org.apache.lucene.codecs; import java.io.Closeable; 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.SegmentWriteState; import org.apache.lucene.index.TermsEnum; @@ -28,8 +28,8 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; /** - * Class that plugs into term dictionaries, such as {@link BlockTreeTermsWriter}, and handles - * writing postings. + * Class that plugs into term dictionaries, such as {@link Lucene90BlockTreeTermsWriter}, and + * handles writing postings. * * @see PostingsReaderBase * @lucene.experimental diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForDeltaUtil.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForDeltaUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForDeltaUtil.java index f8e38da86bb..17f9e42d0ae 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForDeltaUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForUtil.java index cbfdd6579c8..b8f4a49aeef 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/ForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/ForUtil.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java index 3f84280600d..65c866de218 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java @@ -33,7 +33,6 @@ import org.apache.lucene.codecs.VectorFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat; 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.Lucene86SegmentInfoFormat; import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat; @@ -109,7 +108,7 @@ public class Lucene90Codec extends Codec { super("Lucene90"); this.storedFieldsFormat = new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); - this.defaultFormat = new Lucene84PostingsFormat(); + this.defaultFormat = new Lucene90PostingsFormat(); this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsFormat.java new file mode 100644 index 00000000000..e073a2832a9 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsFormat.java @@ -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. + * + *

    Basic idea: + * + *

      + *
    • Packed Blocks and VInt Blocks: + *

      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. + *

      In VInt blocks, integers are encoded as {@link DataOutput#writeVInt VInt}: the block + * size is variable. + *

    • Block structure: + *

      When the postings are long enough, Lucene90PostingsFormat will try to encode most + * integer data as a packed block. + *

      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. + *

      Different kinds of data are always encoded separately into different packed blocks, but + * may possibly be interleaved into the same VInt block. + *

      This strategy is applied to pairs: <document number, frequency>, <position, + * payload length>, <position, offset start, offset length>, and <position, + * payload length, offsetstart, offset length>. + *

    • Skipdata settings: + *

      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. + *

    • Positions, Payloads, and Offsets: + *

      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. + *

      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. + *

      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). + *

      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. + *

    + * + *

    Files and detailed format: + * + *

    + * + * + * + *
    + *
    Term Dictionary + *

    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. + *

    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: + *

      + *
    • PostingsHeader --> Header, PackedBlockSize + *
    • TermMetadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, + * PayFPDelta?, SkipFPDelta? + *
    • Header, --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • PackedBlockSize, SingletonDocID --> {@link DataOutput#writeVInt VInt} + *
    • DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --> {@link + * DataOutput#writeVLong VLong} + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • Header is a {@link CodecUtil#writeIndexHeader IndexHeader} storing the version + * information for the postings. + *
    • 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)}. + *
    • 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. + *
    • 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). + *
    • 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). + *
    • 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). + *
    • 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. + *
    + *
    + * + * + * + *
    + *
    Term Index + *

    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. + *

    + * + * + * + *
    + *
    Frequencies and Skip Data + *

    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. + *

      + *
    • docFile(.doc) --> Header, <TermFreqs, SkipData?>TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermFreqs --> <PackedBlock> PackedDocBlockNum, VIntBlock? + *
    • PackedBlock --> PackedDocDeltaBlock, PackedFreqBlock? + *
    • VIntBlock --> <DocDelta[, + * Freq?]>DocFreq-PackedBlockSize*PackedDocBlockNum + *
    • SkipData --> <<SkipLevelLength, SkipLevel> NumSkipLevels-1, + * SkipLevel>, SkipDatum? + *
    • SkipLevel --> <SkipDatum> TrimmedDocFreq/(PackedBlockSize^(Level + + * 1)) + *
    • SkipDatum --> DocSkip, DocFPSkip, <PosFPSkip, PosBlockOffset, PayLength?, + * PayFPSkip?>?, SkipChildLevelPointer? + *
    • PackedDocDeltaBlock, PackedFreqBlock --> {@link PackedInts PackedInts} + *
    • DocDelta, Freq, DocSkip, DocFPSkip, PosFPSkip, PosBlockOffset, PayByteUpto, PayFPSkip + * --> {@link DataOutput#writeVInt VInt} + *
    • SkipChildLevelPointer --> {@link DataOutput#writeVLong VLong} + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • PackedDocDeltaBlock is theoretically generated from two steps: + *
        + *
      1. Calculate the difference between each document number and previous one, and get + * a d-gaps list (for the first document, use absolute value); + *
      2. For those d-gaps from first one to + * PackedDocBlockNum*PackedBlockSizeth, separately encode as packed + * blocks. + *
      + * If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step. + *
    • VIntBlock stores remaining d-gaps (along with frequencies when possible) with a + * format that encodes DocDelta and Freq: + *

      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. + *

      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: + *

      15, 8, 3 + *

      If frequencies were omitted ({@link IndexOptions#DOCS}) it would be this sequence + * of VInts instead: + *

      7,4 + *

    • PackedDocBlockNum is the number of packed blocks for current term's docids or + * frequencies. In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize) + *
    • 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 + * skipIntervalth, 2*skipIntervalth ... posting in the list. + * However, in Lucene90PostingsFormat, the skip data is saved for + * skipInterval+1th, 2*skipInterval+1th ... posting + * (skipInterval==PackedBlockSize in this case). When DocFreq is multiple of + * PackedBlockSize, MultiLevelSkipListWriter will expect one more skip data than + * Lucene90SkipWriter. + *
    • SkipDatum is the metadata of one skip entry. For the first block (no matter packed or + * VInt), it is omitted. + *
    • DocSkip records the document number of every PackedBlockSizeth 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. + *
    • DocFPSkip records the file offsets of each block (excluding )posting at + * PackedBlockSize+1th, 2*PackedBlockSize+1th ... , 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. + *
    • 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. + *
    • 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 + *
    + *
    + * + * + * + *
    + *
    Positions + *

    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. + *

      + *
    • PosFile(.pos) --> Header, <TermPositions> TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPositions --> <PackedPosDeltaBlock> PackedPosBlockNum, + * VIntBlock? + *
    • VIntBlock --> <PositionDelta[, PayloadLength?], PayloadData?, OffsetDelta?, + * OffsetLength?>PosVIntCount + *
    • PackedPosDeltaBlock --> {@link PackedInts PackedInts} + *
    • PositionDelta, OffsetDelta, OffsetLength --> {@link DataOutput#writeVInt VInt} + *
    • PayloadData --> {@link DataOutput#writeByte byte}PayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • 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. + *
    • PackedPosBlockNum is the number of packed blocks for current term's positions, + * payloads or offsets. In particular, PackedPosBlockNum = + * floor(totalTermFreq/PackedBlockSize) + *
    • PosVIntCount is the number of positions encoded as VInt format. In particular, + * PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize + *
    • The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock + * in chapter Frequencies and Skip Data. + *
    • 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. + *
    • 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): + *

      4, 5, 4 + *

    • 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. + *
    • 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}. + *
    + *
    + * + * + * + *
    + *
    Payloads and Offsets + *

    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. + *

      + *
    • PayFile(.pay): --> Header, <TermPayloads, TermOffsets?> + * TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPayloads --> <PackedPayLengthBlock, SumPayLength, PayData> + * PackedPayBlockNum + *
    • TermOffsets --> <PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock> + * PackedPayBlockNum + *
    • PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --> + * {@link PackedInts PackedInts} + *
    • SumPayLength --> {@link DataOutput#writeVInt VInt} + *
    • PayData --> {@link DataOutput#writeByte byte}SumPayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • The order of TermPayloads/TermOffsets will be the same as TermPositions, note that + * part of payload/offsets are stored in .pos. + *
    • The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is + * the same as PackedFreqBlock in chapter Frequencies and Skip + * Data. While PackedStartDeltaBlock follows a same procedure as + * PackedDocDeltaBlock. + *
    • PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also + * synonym for PackedOffsetBlockNum. + *
    • SumPayLength is the total length of payloads written within one block, should be the + * sum of PayLengths in one packed block. + *
    • PayLength in PackedPayLengthBlock is the length of each payload associated with the + * current position. + *
    + *
    + * + * @lucene.experimental + */ +public final class Lucene90PostingsFormat extends PostingsFormat { + + /** + * Filename extension for document number, frequencies, and skip data. See chapter: Frequencies and Skip Data + */ + public static final String DOC_EXTENSION = "doc"; + + /** Filename extension for positions. See chapter: Positions */ + public static final String POS_EXTENSION = "pos"; + + /** + * Filename extension for payloads and offsets. See chapter: Payloads and + * Offsets + */ + 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; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsReader.java new file mode 100644 index 00000000000..832db5a1233 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsReader.java @@ -0,0 +1,2087 @@ +/* + * 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 static org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.VERSION_START; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat.IntBlockTermState; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Impacts; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SlowImpactsEnum; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; + +/** + * Concrete class that reads docId(maybe frq,pos,offset,payloads) list with postings format. + * + * @lucene.experimental + */ +public final class Lucene90PostingsReader extends PostingsReaderBase { + + private static final long BASE_RAM_BYTES_USED = + RamUsageEstimator.shallowSizeOfInstance(Lucene90PostingsReader.class); + + private final IndexInput docIn; + private final IndexInput posIn; + private final IndexInput payIn; + + private final int version; + + /** Sole constructor. */ + public Lucene90PostingsReader(SegmentReadState state) throws IOException { + boolean success = false; + IndexInput docIn = null; + IndexInput posIn = null; + IndexInput payIn = null; + + // NOTE: these data files are too costly to verify checksum against all the bytes on open, + // but for now we at least verify proper structure of the checksum footer: which looks + // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption + // such as file truncation. + + String docName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene90PostingsFormat.DOC_EXTENSION); + try { + docIn = state.directory.openInput(docName, state.context); + version = + CodecUtil.checkIndexHeader( + docIn, + DOC_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + CodecUtil.retrieveChecksum(docIn); + + if (state.fieldInfos.hasProx()) { + String proxName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene90PostingsFormat.POS_EXTENSION); + posIn = state.directory.openInput(proxName, state.context); + CodecUtil.checkIndexHeader( + posIn, POS_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(posIn); + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene90PostingsFormat.PAY_EXTENSION); + payIn = state.directory.openInput(payName, state.context); + CodecUtil.checkIndexHeader( + payIn, PAY_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(payIn); + } + } + + this.docIn = docIn; + this.posIn = posIn; + this.payIn = payIn; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(docIn, posIn, payIn); + } + } + } + + @Override + public void init(IndexInput termsIn, SegmentReadState state) throws IOException { + // Make sure we are talking to the matching postings writer + CodecUtil.checkIndexHeader( + termsIn, + TERMS_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + final int indexBlockSize = termsIn.readVInt(); + if (indexBlockSize != BLOCK_SIZE) { + throw new IllegalStateException( + "index-time BLOCK_SIZE (" + + indexBlockSize + + ") != read-time BLOCK_SIZE (" + + BLOCK_SIZE + + ")"); + } + } + + /** Read values that have been written using variable-length encoding instead of bit-packing. */ + static void readVIntBlock( + IndexInput docIn, long[] docBuffer, long[] freqBuffer, int num, boolean indexHasFreq) + throws IOException { + if (indexHasFreq) { + for (int i = 0; i < num; i++) { + final int code = docIn.readVInt(); + docBuffer[i] = code >>> 1; + if ((code & 1) != 0) { + freqBuffer[i] = 1; + } else { + freqBuffer[i] = docIn.readVInt(); + } + } + } else { + for (int i = 0; i < num; i++) { + docBuffer[i] = docIn.readVInt(); + } + } + } + + static void prefixSum(long[] buffer, int count, long base) { + buffer[0] += base; + for (int i = 1; i < count; ++i) { + buffer[i] += buffer[i - 1]; + } + } + + static int findFirstGreater(long[] buffer, int target, int from) { + for (int i = from; i < BLOCK_SIZE; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return BLOCK_SIZE; + } + + @Override + public BlockTermState newTermState() { + return new IntBlockTermState(); + } + + @Override + public void close() throws IOException { + IOUtils.close(docIn, posIn, payIn); + } + + @Override + public void decodeTerm( + DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute) + throws IOException { + final IntBlockTermState termState = (IntBlockTermState) _termState; + final boolean fieldHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean fieldHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean fieldHasPayloads = fieldInfo.hasPayloads(); + + if (absolute) { + termState.docStartFP = 0; + termState.posStartFP = 0; + termState.payStartFP = 0; + } + + final long l = in.readVLong(); + if ((l & 0x01) == 0) { + termState.docStartFP += l >>> 1; + if (termState.docFreq == 1) { + termState.singletonDocID = in.readVInt(); + } else { + termState.singletonDocID = -1; + } + } else { + assert absolute == false; + assert termState.singletonDocID != -1; + termState.singletonDocID += BitUtil.zigZagDecode(l >>> 1); + } + + if (fieldHasPositions) { + termState.posStartFP += in.readVLong(); + if (fieldHasOffsets || fieldHasPayloads) { + termState.payStartFP += in.readVLong(); + } + if (termState.totalTermFreq > BLOCK_SIZE) { + termState.lastPosBlockOffset = in.readVLong(); + } else { + termState.lastPosBlockOffset = -1; + } + } + + if (termState.docFreq > BLOCK_SIZE) { + termState.skipOffset = in.readVLong(); + } else { + termState.skipOffset = -1; + } + } + + @Override + public PostingsEnum postings( + FieldInfo fieldInfo, BlockTermState termState, PostingsEnum reuse, int flags) + throws IOException { + + boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false) { + BlockDocsEnum docsEnum; + if (reuse instanceof BlockDocsEnum) { + docsEnum = (BlockDocsEnum) reuse; + if (!docsEnum.canReuse(docIn, fieldInfo)) { + docsEnum = new BlockDocsEnum(fieldInfo); + } + } else { + docsEnum = new BlockDocsEnum(fieldInfo); + } + return docsEnum.reset((IntBlockTermState) termState, flags); + } else { + EverythingEnum everythingEnum; + if (reuse instanceof EverythingEnum) { + everythingEnum = (EverythingEnum) reuse; + if (!everythingEnum.canReuse(docIn, fieldInfo)) { + everythingEnum = new EverythingEnum(fieldInfo); + } + } else { + everythingEnum = new EverythingEnum(fieldInfo); + } + return everythingEnum.reset((IntBlockTermState) termState, flags); + } + } + + @Override + public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) + throws IOException { + if (state.docFreq <= BLOCK_SIZE) { + // no skip data + return new SlowImpactsEnum(postings(fieldInfo, state, null, flags)); + } + + final boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean indexHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean indexHasPayloads = fieldInfo.hasPayloads(); + + if (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false) { + return new BlockImpactsDocsEnum(fieldInfo, (IntBlockTermState) state); + } + + if (indexHasPositions + && PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) + && (indexHasOffsets == false + || PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) == false) + && (indexHasPayloads == false + || PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) == false)) { + return new BlockImpactsPostingsEnum(fieldInfo, (IntBlockTermState) state); + } + + return new BlockImpactsEverythingEnum(fieldInfo, (IntBlockTermState) state, flags); + } + + final class BlockDocsEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + private Lucene90SkipReader skipper; + private boolean skipped; + + final IndexInput startDocIn; + + IndexInput docIn; + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int blockUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long accum; // accumulator for doc deltas + + // Where this term's postings start in the .doc file: + private long docTermStartFP; + + // Where this term's skip data starts (after + // docTermStartFP) in the .doc file (or -1 if there is + // no skip data for this term): + private long skipOffset; + + // docID for next skip point, we won't use skipper if + // target docID is not larger than this + private int nextSkipDoc; + + private boolean needsFreq; // true if the caller actually needs frequencies + // as we read freqBuffer lazily, isFreqsRead shows if freqBuffer are read for the current block + // always true when we don't have freqBuffer (indexHasFreq=false) or don't need freqBuffer + // (needsFreq=false) + private boolean isFreqsRead; + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public BlockDocsEnum(FieldInfo fieldInfo) throws IOException { + this.startDocIn = Lucene90PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && indexHasFreq + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) + && indexHasPos + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) + >= 0) + && indexHasPayloads == fieldInfo.hasPayloads(); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + docFreq = termState.docFreq; + totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; + docTermStartFP = termState.docStartFP; + skipOffset = termState.skipOffset; + singletonDocID = termState.singletonDocID; + if (docFreq > 1) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + docIn.seek(docTermStartFP); + } + + doc = -1; + this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS); + this.isFreqsRead = true; + if (indexHasFreq == false || needsFreq == false) { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + freqBuffer[i] = 1; + } + } + accum = 0; + blockUpto = 0; + nextSkipDoc = BLOCK_SIZE - 1; // we won't skip if target is found in first block + docBufferUpto = BLOCK_SIZE; + skipped = false; + return this; + } + + @Override + public int freq() throws IOException { + if (isFreqsRead == false) { + pforUtil.decode(docIn, freqBuffer); // read freqBuffer for this block + isFreqsRead = true; + } + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() throws IOException { + return -1; + } + + @Override + public int endOffset() throws IOException { + return -1; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + // Check if we skipped reading the previous block of freqBuffer, and if yes, position docIn + // after it + if (isFreqsRead == false) { + pforUtil.skip(docIn); + isFreqsRead = true; + } + + final int left = docFreq - blockUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, accum, docBuffer); + + if (indexHasFreq) { + if (needsFreq) { + isFreqsRead = false; + } else { + pforUtil.skip(docIn); // skip over freqBuffer if we don't need them at all + } + } + blockUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + blockUpto++; + } else { + // Read vInts: + readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq); + prefixSum(docBuffer, left, accum); + docBuffer[left] = NO_MORE_DOCS; + blockUpto += left; + } + accum = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + refillDocs(); // we don't need to load freqBuffer for now (will be loaded later if + // necessary) + } + + doc = (int) docBuffer[docBufferUpto]; + docBufferUpto++; + return doc; + } + + @Override + public int advance(int target) throws IOException { + // current skip docID < docIDs generated from current buffer <= next skip docID + // we don't need to skip if target is buffered already + if (docFreq > BLOCK_SIZE && target > nextSkipDoc) { + + if (skipper == null) { + // Lazy init: first time this enum has ever been used for skipping + skipper = + new Lucene90SkipReader( + docIn.clone(), MAX_SKIP_LEVELS, indexHasPos, indexHasOffsets, indexHasPayloads); + } + + if (!skipped) { + assert skipOffset != -1; + // This is the first time this enum has skipped + // since reset() was called; load the skip data: + skipper.init(docTermStartFP + skipOffset, docTermStartFP, 0, 0, docFreq); + skipped = true; + } + + // always plus one to fix the result, since skip position in Lucene90SkipReader + // is a little different from MultiLevelSkipListReader + final int newDocUpto = skipper.skipTo(target) + 1; + + if (newDocUpto >= blockUpto) { + // Skipper moved + assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto; + blockUpto = newDocUpto; + + // Force to read next block + docBufferUpto = BLOCK_SIZE; + accum = skipper.getDoc(); // actually, this is just lastSkipEntry + docIn.seek(skipper.getDocPointer()); // now point to the block we want to search + // even if freqBuffer were not read from the previous block, we will mark them as read, + // as we don't need to skip the previous block freqBuffer in refillDocs, + // as we have already positioned docIn where in needs to be. + isFreqsRead = true; + } + // next time we call advance, this is used to + // foresee whether skipper is necessary. + nextSkipDoc = skipper.getNextSkipDoc(); + } + if (docBufferUpto == BLOCK_SIZE) { + refillDocs(); + } + + // Now scan... this is an inlined/pared down version + // of nextDoc(): + long doc; + while (true) { + doc = docBuffer[docBufferUpto]; + + if (doc >= target) { + break; + } + ++docBufferUpto; + } + + docBufferUpto++; + return this.doc = (int) doc; + } + + @Override + public long cost() { + return docFreq; + } + } + + // Also handles payloads + offsets + final class EverythingEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE + 1]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private final long[] payloadLengthBuffer; + private final long[] offsetStartDeltaBuffer; + private final long[] offsetLengthBuffer; + + private byte[] payloadBytes; + private int payloadByteUpto; + private int payloadLength; + + private int lastStartOffset; + private int startOffset; + private int endOffset; + + private int docBufferUpto; + private int posBufferUpto; + + private Lucene90SkipReader skipper; + private boolean skipped; + + final IndexInput startDocIn; + + IndexInput docIn; + final IndexInput posIn; + final IndexInput payIn; + final BytesRef payload; + + final boolean indexHasOffsets; + final boolean indexHasPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // number of positions in this posting list + private int blockUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long accum; // accumulator for doc deltas + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private int posPendingCount; + + // Lazy pos seek: if != -1 then we must seek to this FP + // before reading positions: + private long posPendingFP; + + // Lazy pay seek: if != -1 then we must seek to this FP + // before reading payloads/offsets: + private long payPendingFP; + + // Where this term's postings start in the .doc file: + private long docTermStartFP; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // Where this term's payloads/offsets start in the .pay + // file: + private long payTermStartFP; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + // Where this term's skip data starts (after + // docTermStartFP) in the .doc file (or -1 if there is + // no skip data for this term): + private long skipOffset; + + private int nextSkipDoc; + + private boolean needsOffsets; // true if we actually need offsets + private boolean needsPayloads; // true if we actually need payloads + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public EverythingEnum(FieldInfo fieldInfo) throws IOException { + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + + this.startDocIn = Lucene90PostingsReader.this.docIn; + this.docIn = null; + this.posIn = Lucene90PostingsReader.this.posIn.clone(); + if (indexHasOffsets || indexHasPayloads) { + this.payIn = Lucene90PostingsReader.this.payIn.clone(); + } else { + this.payIn = null; + } + if (indexHasOffsets) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + startOffset = -1; + endOffset = -1; + } + + if (indexHasPayloads) { + payloadLengthBuffer = new long[BLOCK_SIZE]; + payloadBytes = new byte[128]; + payload = new BytesRef(); + } else { + payloadLengthBuffer = null; + payloadBytes = null; + payload = null; + } + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && indexHasOffsets + == (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0) + && indexHasPayloads == fieldInfo.hasPayloads(); + } + + public EverythingEnum reset(IntBlockTermState termState, int flags) throws IOException { + docFreq = termState.docFreq; + docTermStartFP = termState.docStartFP; + posTermStartFP = termState.posStartFP; + payTermStartFP = termState.payStartFP; + skipOffset = termState.skipOffset; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + docIn.seek(docTermStartFP); + } + posPendingFP = posTermStartFP; + payPendingFP = payTermStartFP; + posPendingCount = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + this.needsOffsets = PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS); + this.needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS); + + doc = -1; + accum = 0; + blockUpto = 0; + if (docFreq > BLOCK_SIZE) { + nextSkipDoc = BLOCK_SIZE - 1; // we won't skip if target is found in first block + } else { + nextSkipDoc = NO_MORE_DOCS; // not enough docs for skipping + } + docBufferUpto = BLOCK_SIZE; + skipped = false; + return this; + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - blockUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, accum, docBuffer); + pforUtil.decode(docIn, freqBuffer); + blockUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + blockUpto++; + } else { + readVIntBlock(docIn, docBuffer, freqBuffer, left, true); + prefixSum(docBuffer, left, accum); + docBuffer[left] = NO_MORE_DOCS; + blockUpto += left; + } + accum = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + int offsetLength = 0; + payloadByteUpto = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + payloadLengthBuffer[i] = payloadLength; + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + if (payloadByteUpto + payloadLength > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength); + } + posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength); + payloadByteUpto += payloadLength; + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + offsetLength = posIn.readVInt(); + } + offsetStartDeltaBuffer[i] = deltaCode >>> 1; + offsetLengthBuffer[i] = offsetLength; + } + } + payloadByteUpto = 0; + } else { + pforUtil.decode(posIn, posDeltaBuffer); + + if (indexHasPayloads) { + if (needsPayloads) { + pforUtil.decode(payIn, payloadLengthBuffer); + int numBytes = payIn.readVInt(); + + if (numBytes > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, numBytes); + } + payIn.readBytes(payloadBytes, 0, numBytes); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over lengths + int numBytes = payIn.readVInt(); // read length of payloadBytes + payIn.seek(payIn.getFilePointer() + numBytes); // skip over payloadBytes + } + payloadByteUpto = 0; + } + + if (indexHasOffsets) { + if (needsOffsets) { + pforUtil.decode(payIn, offsetStartDeltaBuffer); + pforUtil.decode(payIn, offsetLengthBuffer); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over starts + pforUtil.skip(payIn); // skip over lengths + } + } + } + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + refillDocs(); + } + + doc = (int) docBuffer[docBufferUpto]; + freq = (int) freqBuffer[docBufferUpto]; + posPendingCount += freq; + docBufferUpto++; + + position = 0; + lastStartOffset = 0; + return doc; + } + + @Override + public int advance(int target) throws IOException { + if (target > nextSkipDoc) { + if (skipper == null) { + // Lazy init: first time this enum has ever been used for skipping + skipper = + new Lucene90SkipReader( + docIn.clone(), MAX_SKIP_LEVELS, true, indexHasOffsets, indexHasPayloads); + } + + if (!skipped) { + assert skipOffset != -1; + // This is the first time this enum has skipped + // since reset() was called; load the skip data: + skipper.init( + docTermStartFP + skipOffset, docTermStartFP, posTermStartFP, payTermStartFP, docFreq); + skipped = true; + } + + final int newDocUpto = skipper.skipTo(target) + 1; + + if (newDocUpto > blockUpto - BLOCK_SIZE + docBufferUpto) { + // Skipper moved + assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto; + blockUpto = newDocUpto; + + // Force to read next block + docBufferUpto = BLOCK_SIZE; + accum = skipper.getDoc(); + docIn.seek(skipper.getDocPointer()); + posPendingFP = skipper.getPosPointer(); + payPendingFP = skipper.getPayPointer(); + posPendingCount = skipper.getPosBufferUpto(); + lastStartOffset = 0; // new document + payloadByteUpto = skipper.getPayloadByteUpto(); + } + nextSkipDoc = skipper.getNextSkipDoc(); + } + if (docBufferUpto == BLOCK_SIZE) { + refillDocs(); + } + + // Now scan: + long doc; + while (true) { + doc = docBuffer[docBufferUpto]; + freq = (int) freqBuffer[docBufferUpto]; + posPendingCount += freq; + docBufferUpto++; + + if (doc >= target) { + break; + } + } + + position = 0; + lastStartOffset = 0; + return this.doc = (int) doc; + } + + // TODO: in theory we could avoid loading frq block + // when not needed, ie, use skip data to load how far to + // seek the pos pointer ... instead of having to load frq + // blocks only to sum up how many positions to skip + private void skipPositions() throws IOException { + // Skip positions now: + int toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + int end = posBufferUpto + toSkip; + while (posBufferUpto < end) { + if (indexHasPayloads) { + payloadByteUpto += payloadLengthBuffer[posBufferUpto]; + } + posBufferUpto++; + } + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + pforUtil.skip(posIn); + + if (indexHasPayloads) { + // Skip payloadLength block: + pforUtil.skip(payIn); + + // Skip payloadBytes block: + int numBytes = payIn.readVInt(); + payIn.seek(payIn.getFilePointer() + numBytes); + } + + if (indexHasOffsets) { + pforUtil.skip(payIn); + pforUtil.skip(payIn); + } + toSkip -= BLOCK_SIZE; + } + refillPositions(); + payloadByteUpto = 0; + posBufferUpto = 0; + while (posBufferUpto < toSkip) { + if (indexHasPayloads) { + payloadByteUpto += payloadLengthBuffer[posBufferUpto]; + } + posBufferUpto++; + } + } + + position = 0; + lastStartOffset = 0; + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingFP != -1) { + posIn.seek(posPendingFP); + posPendingFP = -1; + + if (payPendingFP != -1 && payIn != null) { + payIn.seek(payPendingFP); + payPendingFP = -1; + } + + // Force buffer refill: + posBufferUpto = BLOCK_SIZE; + } + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + if (indexHasPayloads) { + payloadLength = (int) payloadLengthBuffer[posBufferUpto]; + payload.bytes = payloadBytes; + payload.offset = payloadByteUpto; + payload.length = payloadLength; + payloadByteUpto += payloadLength; + } + + if (indexHasOffsets) { + startOffset = lastStartOffset + (int) offsetStartDeltaBuffer[posBufferUpto]; + endOffset = startOffset + (int) offsetLengthBuffer[posBufferUpto]; + lastStartOffset = startOffset; + } + + posBufferUpto++; + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return startOffset; + } + + @Override + public int endOffset() { + return endOffset; + } + + @Override + public BytesRef getPayload() { + if (payloadLength == 0) { + return null; + } else { + return payload; + } + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsDocsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + private final Lucene90ScoreSkipReader skipper; + + final IndexInput docIn; + + final boolean indexHasFreqs; + + private int docFreq; // number of docs in this posting list + private int blockUpto; // number of documents in or before the current block + private int doc; // doc we last read + private long accum; // accumulator for doc deltas + + private int nextSkipDoc = -1; + + private long seekTo = -1; + + // as we read freqBuffer lazily, isFreqsRead shows if freqBuffer are read for the current block + // always true when we don't have freqBuffer (indexHasFreq=false) or don't need freqBuffer + // (needsFreq=false) + private boolean isFreqsRead; + + public BlockImpactsDocsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + indexHasFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + final boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean indexHasPayloads = fieldInfo.hasPayloads(); + + this.docIn = Lucene90PostingsReader.this.docIn.clone(); + + docFreq = termState.docFreq; + docIn.seek(termState.docStartFP); + + doc = -1; + accum = 0; + blockUpto = 0; + docBufferUpto = BLOCK_SIZE; + + skipper = + new Lucene90ScoreSkipReader( + docIn.clone(), MAX_SKIP_LEVELS, indexHasPositions, indexHasOffsets, indexHasPayloads); + skipper.init( + termState.docStartFP + termState.skipOffset, + termState.docStartFP, + termState.posStartFP, + termState.payStartFP, + docFreq); + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + this.isFreqsRead = true; + if (indexHasFreqs == false) { + Arrays.fill(freqBuffer, 1L); + } + } + + @Override + public int freq() throws IOException { + if (isFreqsRead == false) { + pforUtil.decode(docIn, freqBuffer); // read freqBuffer for this block + isFreqsRead = true; + } + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + // Check if we skipped reading the previous block of freqBuffer, and if yes, position docIn + // after it + if (isFreqsRead == false) { + pforUtil.skip(docIn); + isFreqsRead = true; + } + + final int left = docFreq - blockUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, accum, docBuffer); + if (indexHasFreqs) { + pforUtil.decode(docIn, freqBuffer); + } + blockUpto += BLOCK_SIZE; + } else { + readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreqs); + prefixSum(docBuffer, left, accum); + docBuffer[left] = NO_MORE_DOCS; + blockUpto += left; + } + accum = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > nextSkipDoc) { + // always plus one to fix the result, since skip position in Lucene90SkipReader + // is a little different from MultiLevelSkipListReader + final int newDocUpto = skipper.skipTo(target) + 1; + + if (newDocUpto >= blockUpto) { + // Skipper moved + assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto; + blockUpto = newDocUpto; + + // Force to read next block + docBufferUpto = BLOCK_SIZE; + accum = skipper.getDoc(); + seekTo = skipper.getDocPointer(); // delay the seek + } + // next time we call advance, this is used to + // foresee whether skipper is necessary. + nextSkipDoc = skipper.getNextSkipDoc(); + } + assert nextSkipDoc >= target; + } + + @Override + public Impacts getImpacts() throws IOException { + advanceShallow(doc); + return skipper.getImpacts(); + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target > nextSkipDoc) { + advanceShallow(target); + } + if (docBufferUpto == BLOCK_SIZE) { + if (seekTo >= 0) { + docIn.seek(seekTo); + isFreqsRead = true; // reset isFreqsRead + seekTo = -1; + } + refillDocs(); + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + this.doc = (int) docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsPostingsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + private int posBufferUpto; + + private final Lucene90ScoreSkipReader skipper; + + final IndexInput docIn; + final IndexInput posIn; + + final boolean indexHasOffsets; + final boolean indexHasPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // number of positions in this posting list + private int docUpto; // how many docs we've read + private int doc; // doc we last read + private long accum; // accumulator for doc deltas + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private int posPendingCount; + + // Lazy pos seek: if != -1 then we must seek to this FP + // before reading positions: + private long posPendingFP; + + // Where this term's postings start in the .doc file: + private long docTermStartFP; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // Where this term's payloads/offsets start in the .pay + // file: + private long payTermStartFP; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + private int nextSkipDoc = -1; + + private long seekTo = -1; + + public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + + this.docIn = Lucene90PostingsReader.this.docIn.clone(); + + this.posIn = Lucene90PostingsReader.this.posIn.clone(); + + docFreq = termState.docFreq; + docTermStartFP = termState.docStartFP; + posTermStartFP = termState.posStartFP; + payTermStartFP = termState.payStartFP; + totalTermFreq = termState.totalTermFreq; + docIn.seek(docTermStartFP); + posPendingFP = posTermStartFP; + posPendingCount = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + doc = -1; + accum = 0; + docUpto = 0; + docBufferUpto = BLOCK_SIZE; + + skipper = + new Lucene90ScoreSkipReader( + docIn.clone(), MAX_SKIP_LEVELS, true, indexHasOffsets, indexHasPayloads); + skipper.init( + docTermStartFP + termState.skipOffset, + docTermStartFP, + posTermStartFP, + payTermStartFP, + docFreq); + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, accum, docBuffer); + pforUtil.decode(docIn, freqBuffer); + } else { + readVIntBlock(docIn, docBuffer, freqBuffer, left, true); + prefixSum(docBuffer, left, accum); + docBuffer[left] = NO_MORE_DOCS; + } + accum = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + posIn.seek(posIn.getFilePointer() + payloadLength); + } + } else { + posDeltaBuffer[i] = code; + } + if (indexHasOffsets) { + if ((posIn.readVInt() & 1) != 0) { + // offset length changed + posIn.readVInt(); + } + } + } + } else { + pforUtil.decode(posIn, posDeltaBuffer); + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > nextSkipDoc) { + // always plus one to fix the result, since skip position in Lucene90SkipReader + // is a little different from MultiLevelSkipListReader + final int newDocUpto = skipper.skipTo(target) + 1; + + if (newDocUpto > docUpto) { + // Skipper moved + assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto; + docUpto = newDocUpto; + + // Force to read next block + docBufferUpto = BLOCK_SIZE; + accum = skipper.getDoc(); + posPendingFP = skipper.getPosPointer(); + posPendingCount = skipper.getPosBufferUpto(); + seekTo = skipper.getDocPointer(); // delay the seek + } + // next time we call advance, this is used to + // foresee whether skipper is necessary. + nextSkipDoc = skipper.getNextSkipDoc(); + } + assert nextSkipDoc >= target; + } + + @Override + public Impacts getImpacts() throws IOException { + advanceShallow(doc); + return skipper.getImpacts(); + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target > nextSkipDoc) { + advanceShallow(target); + } + if (docBufferUpto == BLOCK_SIZE) { + if (seekTo >= 0) { + docIn.seek(seekTo); + seekTo = -1; + } + refillDocs(); + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + if (next == BLOCK_SIZE) { + return doc = NO_MORE_DOCS; + } + this.doc = (int) docBuffer[next]; + this.freq = (int) freqBuffer[next]; + for (int i = docBufferUpto; i <= next; ++i) { + posPendingCount += freqBuffer[i]; + } + docUpto += next - docBufferUpto + 1; + docBufferUpto = next + 1; + position = 0; + return doc; + } + + // TODO: in theory we could avoid loading frq block + // when not needed, ie, use skip data to load how far to + // seek the pos pointer ... instead of having to load frq + // blocks only to sum up how many positions to skip + private void skipPositions() throws IOException { + // Skip positions now: + int toSkip = posPendingCount - freq; + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + posBufferUpto += toSkip; + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + pforUtil.skip(posIn); + toSkip -= BLOCK_SIZE; + } + refillPositions(); + posBufferUpto = toSkip; + } + + position = 0; + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingFP != -1) { + posIn.seek(posPendingFP); + posPendingFP = -1; + + // Force buffer refill: + posBufferUpto = BLOCK_SIZE; + } + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto++]; + + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsEverythingEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private final long[] payloadLengthBuffer; + private final long[] offsetStartDeltaBuffer; + private final long[] offsetLengthBuffer; + + private byte[] payloadBytes; + private int payloadByteUpto; + private int payloadLength; + + private int lastStartOffset; + private int startOffset = -1; + private int endOffset = -1; + + private int docBufferUpto; + private int posBufferUpto; + + private final Lucene90ScoreSkipReader skipper; + + final IndexInput docIn; + final IndexInput posIn; + final IndexInput payIn; + final BytesRef payload; + + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // number of positions in this posting list + private int docUpto; // how many docs we've read + private int posDocUpTo; // for how many docs we've read positions, offsets, and payloads + private int doc; // doc we last read + private long accum; // accumulator for doc deltas + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private int posPendingCount; + + // Lazy pos seek: if != -1 then we must seek to this FP + // before reading positions: + private long posPendingFP; + + // Lazy pay seek: if != -1 then we must seek to this FP + // before reading payloads/offsets: + private long payPendingFP; + + // Where this term's postings start in the .doc file: + private long docTermStartFP; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // Where this term's payloads/offsets start in the .pay + // file: + private long payTermStartFP; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + private int nextSkipDoc = -1; + + private final boolean needsPositions; + private final boolean needsOffsets; // true if we actually need offsets + private final boolean needsPayloads; // true if we actually need payloads + + private boolean + isFreqsRead; // shows if freqBuffer for the current doc block are read into freqBuffer + + private long seekTo = -1; + + public BlockImpactsEverythingEnum(FieldInfo fieldInfo, IntBlockTermState termState, int flags) + throws IOException { + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + + needsPositions = PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS); + needsOffsets = PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS); + needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS); + + this.docIn = Lucene90PostingsReader.this.docIn.clone(); + + if (indexHasPos && needsPositions) { + this.posIn = Lucene90PostingsReader.this.posIn.clone(); + } else { + this.posIn = null; + } + + if ((indexHasOffsets && needsOffsets) || (indexHasPayloads && needsPayloads)) { + this.payIn = Lucene90PostingsReader.this.payIn.clone(); + } else { + this.payIn = null; + } + + if (indexHasOffsets) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + startOffset = -1; + endOffset = -1; + } + + if (indexHasPayloads) { + payloadLengthBuffer = new long[BLOCK_SIZE]; + payloadBytes = new byte[128]; + payload = new BytesRef(); + } else { + payloadLengthBuffer = null; + payloadBytes = null; + payload = null; + } + + docFreq = termState.docFreq; + docTermStartFP = termState.docStartFP; + posTermStartFP = termState.posStartFP; + payTermStartFP = termState.payStartFP; + totalTermFreq = termState.totalTermFreq; + docIn.seek(docTermStartFP); + posPendingFP = posTermStartFP; + payPendingFP = payTermStartFP; + posPendingCount = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + doc = -1; + accum = 0; + docUpto = 0; + posDocUpTo = 0; + isFreqsRead = true; + docBufferUpto = BLOCK_SIZE; + + skipper = + new Lucene90ScoreSkipReader( + docIn.clone(), MAX_SKIP_LEVELS, indexHasPos, indexHasOffsets, indexHasPayloads); + skipper.init( + docTermStartFP + termState.skipOffset, + docTermStartFP, + posTermStartFP, + payTermStartFP, + docFreq); + + if (indexHasFreq == false) { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + freqBuffer[i] = 1; + } + } + } + + @Override + public int freq() throws IOException { + if (indexHasFreq && (isFreqsRead == false)) { + pforUtil.decode(docIn, freqBuffer); // read freqBuffer for this block + isFreqsRead = true; + } + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + if (indexHasFreq) { + if (isFreqsRead == false) { // previous freq block was not read + // check if we need to load the previous freq block to catch up on positions or we can + // skip it + if (indexHasPos && needsPositions && (posDocUpTo < docUpto)) { + pforUtil.decode(docIn, freqBuffer); // load the previous freq block + } else { + pforUtil.skip(docIn); // skip it + } + isFreqsRead = true; + } + if (indexHasPos && needsPositions) { + while (posDocUpTo + < docUpto) { // catch on positions, bring posPendingCount upto the current doc + posPendingCount += freqBuffer[docBufferUpto - (docUpto - posDocUpTo)]; + posDocUpTo++; + } + } + } + + final int left = docFreq - docUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, accum, docBuffer); + if (indexHasFreq) { + isFreqsRead = + false; // freq block will be loaded lazily when necessary, we don't load it here + } + } else { + readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq); + prefixSum(docBuffer, left, accum); + docBuffer[left] = NO_MORE_DOCS; + } + accum = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + int offsetLength = 0; + payloadByteUpto = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + payloadLengthBuffer[i] = payloadLength; + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + if (payloadByteUpto + payloadLength > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength); + } + posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength); + payloadByteUpto += payloadLength; + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + offsetLength = posIn.readVInt(); + } + offsetStartDeltaBuffer[i] = deltaCode >>> 1; + offsetLengthBuffer[i] = offsetLength; + } + } + payloadByteUpto = 0; + } else { + pforUtil.decode(posIn, posDeltaBuffer); + + if (indexHasPayloads && payIn != null) { + if (needsPayloads) { + pforUtil.decode(payIn, payloadLengthBuffer); + int numBytes = payIn.readVInt(); + + if (numBytes > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, numBytes); + } + payIn.readBytes(payloadBytes, 0, numBytes); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over lengths + int numBytes = payIn.readVInt(); // read length of payloadBytes + payIn.seek(payIn.getFilePointer() + numBytes); // skip over payloadBytes + } + payloadByteUpto = 0; + } + + if (indexHasOffsets && payIn != null) { + if (needsOffsets) { + pforUtil.decode(payIn, offsetStartDeltaBuffer); + pforUtil.decode(payIn, offsetLengthBuffer); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + pforUtil.skip(payIn); // skip over starts + pforUtil.skip(payIn); // skip over lengths + } + } + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > nextSkipDoc) { + // always plus one to fix the result, since skip position in Lucene90SkipReader + // is a little different from MultiLevelSkipListReader + final int newDocUpto = skipper.skipTo(target) + 1; + + if (newDocUpto > docUpto) { + // Skipper moved + assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto; + docUpto = newDocUpto; + posDocUpTo = docUpto; + + // Force to read next block + docBufferUpto = BLOCK_SIZE; + accum = skipper.getDoc(); + posPendingFP = skipper.getPosPointer(); + payPendingFP = skipper.getPayPointer(); + posPendingCount = skipper.getPosBufferUpto(); + lastStartOffset = 0; // new document + payloadByteUpto = skipper.getPayloadByteUpto(); // actually, this is just lastSkipEntry + seekTo = skipper.getDocPointer(); // delay the seek + } + // next time we call advance, this is used to + // foresee whether skipper is necessary. + nextSkipDoc = skipper.getNextSkipDoc(); + } + assert nextSkipDoc >= target; + } + + @Override + public Impacts getImpacts() throws IOException { + advanceShallow(doc); + return skipper.getImpacts(); + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target > nextSkipDoc) { + advanceShallow(target); + } + if (docBufferUpto == BLOCK_SIZE) { + if (seekTo >= 0) { + docIn.seek(seekTo); + seekTo = -1; + isFreqsRead = true; // reset isFreqsRead + } + refillDocs(); + } + + // Now scan: + long doc; + while (true) { + doc = docBuffer[docBufferUpto]; + docBufferUpto++; + docUpto++; + + if (doc >= target) { + break; + } + + if (docBufferUpto == BLOCK_SIZE) { + return this.doc = NO_MORE_DOCS; + } + } + position = 0; + lastStartOffset = 0; + + return this.doc = (int) doc; + } + + // TODO: in theory we could avoid loading frq block + // when not needed, ie, use skip data to load how far to + // seek the pos pointer ... instead of having to load frq + // blocks only to sum up how many positions to skip + private void skipPositions() throws IOException { + // Skip positions now: + int toSkip = posPendingCount - (int) freqBuffer[docBufferUpto - 1]; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + int end = posBufferUpto + toSkip; + while (posBufferUpto < end) { + if (indexHasPayloads) { + payloadByteUpto += payloadLengthBuffer[posBufferUpto]; + } + posBufferUpto++; + } + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + pforUtil.skip(posIn); + + if (indexHasPayloads && payIn != null) { + // Skip payloadLength block: + pforUtil.skip(payIn); + + // Skip payloadBytes block: + int numBytes = payIn.readVInt(); + payIn.seek(payIn.getFilePointer() + numBytes); + } + + if (indexHasOffsets && payIn != null) { + pforUtil.skip(payIn); + pforUtil.skip(payIn); + } + toSkip -= BLOCK_SIZE; + } + refillPositions(); + payloadByteUpto = 0; + posBufferUpto = 0; + while (posBufferUpto < toSkip) { + if (indexHasPayloads) { + payloadByteUpto += payloadLengthBuffer[posBufferUpto]; + } + posBufferUpto++; + } + } + + position = 0; + lastStartOffset = 0; + } + + @Override + public int nextPosition() throws IOException { + if (indexHasPos == false || needsPositions == false) { + return -1; + } + + if (isFreqsRead == false) { + pforUtil.decode(docIn, freqBuffer); // read freqBuffer for this docs block + isFreqsRead = true; + } + while (posDocUpTo < docUpto) { // bring posPendingCount upto the current doc + posPendingCount += freqBuffer[docBufferUpto - (docUpto - posDocUpTo)]; + posDocUpTo++; + } + + assert posPendingCount > 0; + + if (posPendingFP != -1) { + posIn.seek(posPendingFP); + posPendingFP = -1; + + if (payPendingFP != -1 && payIn != null) { + payIn.seek(payPendingFP); + payPendingFP = -1; + } + + // Force buffer refill: + posBufferUpto = BLOCK_SIZE; + } + + if (posPendingCount > freqBuffer[docBufferUpto - 1]) { + skipPositions(); + posPendingCount = (int) freqBuffer[docBufferUpto - 1]; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + if (indexHasPayloads) { + payloadLength = (int) payloadLengthBuffer[posBufferUpto]; + payload.bytes = payloadBytes; + payload.offset = payloadByteUpto; + payload.length = payloadLength; + payloadByteUpto += payloadLength; + } + + if (indexHasOffsets && needsOffsets) { + startOffset = lastStartOffset + (int) offsetStartDeltaBuffer[posBufferUpto]; + endOffset = startOffset + (int) offsetLengthBuffer[posBufferUpto]; + lastStartOffset = startOffset; + } + + posBufferUpto++; + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return startOffset; + } + + @Override + public int endOffset() { + return endOffset; + } + + @Override + public BytesRef getPayload() { + if (payloadLength == 0) { + return null; + } else { + return payload; + } + } + + @Override + public long cost() { + return docFreq; + } + } + + @Override + public long ramBytesUsed() { + return BASE_RAM_BYTES_USED; + } + + @Override + public void checkIntegrity() throws IOException { + if (docIn != null) { + CodecUtil.checksumEntireFile(docIn); + } + if (posIn != null) { + CodecUtil.checksumEntireFile(posIn); + } + if (payIn != null) { + CodecUtil.checksumEntireFile(payIn); + } + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "(positions=" + + (posIn != null) + + ",payloads=" + + (payIn != null) + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsWriter.java new file mode 100644 index 00000000000..d26cc4dcbf6 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90PostingsWriter.java @@ -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. + * + *

    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; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90ScoreSkipReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90ScoreSkipReader.java new file mode 100644 index 00000000000..44789a98334 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90ScoreSkipReader.java @@ -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 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 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; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipReader.java new file mode 100644 index 00000000000..e05f5ce1f8a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipReader.java @@ -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. + * + *

    Although this skipper uses MultiLevelSkipListReader as an interface, its definition of skip + * position will be a little different. + * + *

    For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6, + * + *

    + * 0 1 2 3 4 5
    + * d d d d d d    (posting list)
    + *     ^     ^    (skip point in MultiLeveSkipWriter)
    + *       ^        (skip point in Lucene90SkipWriter)
    + * 
    + * + *

    In this case, MultiLevelSkipListReader will use the last document as a skip point, while + * Lucene90SkipReader should assume no skip point will comes. + * + *

    If we use the interface directly in Lucene90SkipReader, it may silly try to read another skip + * data after the only skip point is loaded. + * + *

    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 + * + *

    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. + * + *

    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()); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipWriter.java new file mode 100644 index 00000000000..48deafa4225 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90SkipWriter.java @@ -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. + * + *

    Assume that docFreq = 28, skipInterval = blockSize = 12 + * + *

    + *  |       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)
    + * 
    + * + *

    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). + * + *

    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 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; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/PForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/PForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/PForUtil.java index 3a24ea86cd7..17af2e5118e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/PForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/PForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import java.io.IOException; import java.util.Arrays; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/CompressionAlgorithm.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/CompressionAlgorithm.java index 1cfcda207fd..de24dc3fb05 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/CompressionAlgorithm.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.codecs.lucene90.blocktree; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java new file mode 100644 index 00000000000..cb396b713ae --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java @@ -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 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 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 + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnum.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnum.java index 5404a63bc13..3afbc51b2fb 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnum.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.codecs.lucene90.blocktree; import java.io.IOException; import org.apache.lucene.index.BaseTermsEnum; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnumFrame.java new file mode 100644 index 00000000000..ff7e58b11b4 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/IntersectTermsEnumFrame.java @@ -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 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; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java new file mode 100644 index 00000000000..9e9e3a8d749 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java @@ -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). + * + *

    NOTE: this terms dictionary supports min/maxItemsPerBlock during indexing to control + * how much memory the terms index uses. + * + *

    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. + * + *

    Use {@link org.apache.lucene.index.CheckIndex} with the -verbose option to see + * summary statistics on the blocks in the dictionary. + * + *

    See {@link Lucene90BlockTreeTermsWriter}. + * + * @lucene.experimental + */ +public final class Lucene90BlockTreeTermsReader extends FieldsProducer { + + static final Outputs 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 fieldMap; + private final List 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 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 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 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 getChildResources() { + List 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 + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java index 985125a3db0..11a02e344e8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.codecs.lucene90.blocktree; import java.io.IOException; import java.util.ArrayList; @@ -192,20 +192,20 @@ import org.apache.lucene.util.fst.Util; * byte of each sub-block, and its file pointer. * * - * @see BlockTreeTermsReader + * @see Lucene90BlockTreeTermsReader * @lucene.experimental */ -public final class BlockTreeTermsWriter extends FieldsConsumer { +public final class Lucene90BlockTreeTermsWriter extends FieldsConsumer { /** * 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; /** * 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; @@ -231,7 +231,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer { * minItemsPerBlock and maxItemsPerBlock, though in some cases the blocks may be smaller than the * min. */ - public BlockTreeTermsWriter( + public Lucene90BlockTreeTermsWriter( SegmentWriteState state, PostingsWriterBase postingsWriter, int minItemsInBlock, @@ -248,15 +248,17 @@ public final class BlockTreeTermsWriter extends FieldsConsumer { final String termsName = 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); boolean success = false; IndexOutput metaOut = null, indexOut = null; try { CodecUtil.writeIndexHeader( termsOut, - BlockTreeTermsReader.TERMS_CODEC_NAME, - BlockTreeTermsReader.VERSION_CURRENT, + Lucene90BlockTreeTermsReader.TERMS_CODEC_NAME, + Lucene90BlockTreeTermsReader.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); @@ -264,12 +266,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer { IndexFileNames.segmentFileName( state.segmentInfo.name, state.segmentSuffix, - BlockTreeTermsReader.TERMS_INDEX_EXTENSION); + Lucene90BlockTreeTermsReader.TERMS_INDEX_EXTENSION); indexOut = state.directory.createOutput(indexName, state.context); CodecUtil.writeIndexHeader( indexOut, - BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, - BlockTreeTermsReader.VERSION_CURRENT, + Lucene90BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, + Lucene90BlockTreeTermsReader.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); // segment = state.segmentInfo.name; @@ -278,12 +280,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer { IndexFileNames.segmentFileName( state.segmentInfo.name, state.segmentSuffix, - BlockTreeTermsReader.TERMS_META_EXTENSION); + Lucene90BlockTreeTermsReader.TERMS_META_EXTENSION); metaOut = state.directory.createOutput(metaName, state.context); CodecUtil.writeIndexHeader( metaOut, - BlockTreeTermsReader.TERMS_META_CODEC_NAME, - BlockTreeTermsReader.VERSION_CURRENT, + Lucene90BlockTreeTermsReader.TERMS_META_CODEC_NAME, + Lucene90BlockTreeTermsReader.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); @@ -359,8 +361,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer { static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) { assert fp < (1L << 62); return (fp << 2) - | (hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) - | (isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); + | (hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + | (isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); } private static class PendingEntry { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java similarity index 94% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java index b5834788815..0e865061cf0 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.codecs.lucene90.blocktree; import java.io.IOException; import java.io.PrintStream; @@ -237,11 +237,11 @@ final class SegmentTermsEnum extends BaseTermsEnum { throws IOException { scratchReader.reset(frameData.bytes, frameData.offset, frameData.length); 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); - f.hasTerms = (code & BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0; + f.hasTerms = (code & Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0; 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) { f.setFloorData(scratchReader, frameData); } @@ -394,8 +394,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF); - if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { - output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); } if (arc.isFinal()) { lastFrame = stack[1 + lastFrame.ord]; @@ -491,7 +491,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { // term.length = 0; targetUpto = 0; currentFrame = - pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); + pushFrame( + arc, Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); } // if (DEBUG) { @@ -553,8 +554,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { term.setByteAt(targetUpto, (byte) targetLabel); // Aggregate output as we go: assert arc.output() != null; - if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { - output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); } // if (DEBUG) { @@ -568,7 +569,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { currentFrame = pushFrame( arc, - BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), + Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto); // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + // currentFrame.hasTerms); @@ -684,8 +685,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { // seek; but, often the FST doesn't have any // shared bytes (but this could change if we // reverse vLong byte order) - if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { - output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); } if (arc.isFinal()) { lastFrame = stack[1 + lastFrame.ord]; @@ -775,7 +776,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { // term.length = 0; targetUpto = 0; currentFrame = - pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); + pushFrame( + arc, Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0); } // if (DEBUG) { @@ -837,8 +839,8 @@ final class SegmentTermsEnum extends BaseTermsEnum { arc = nextArc; // Aggregate output as we go: assert arc.output() != null; - if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) { - output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); + if (arc.output() != Lucene90BlockTreeTermsReader.NO_OUTPUT) { + output = Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output()); } // if (DEBUG) { @@ -852,7 +854,7 @@ final class SegmentTermsEnum extends BaseTermsEnum { currentFrame = pushFrame( arc, - BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), + Lucene90BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto); // if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + // currentFrame.hasTerms); @@ -919,9 +921,9 @@ final class SegmentTermsEnum extends BaseTermsEnum { + " isFloor=" + f.isFloor + " code=" - + ((f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) - + (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) - + (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + + ((f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + + (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + + (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" @@ -949,9 +951,9 @@ final class SegmentTermsEnum extends BaseTermsEnum { + " isFloor=" + f.isFloor + " code=" - + ((f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) - + (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) - + (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + + ((f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + + (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + + (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" @@ -980,9 +982,9 @@ final class SegmentTermsEnum extends BaseTermsEnum { new ByteArrayDataInput(output.bytes, output.offset, output.length); final long codeOrig = reader.readVLong(); final long code = - (f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) - | (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) - | (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); + (f.fp << Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + | (f.hasTerms ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) + | (f.isFloor ? Lucene90BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0); if (codeOrig != code) { out.println( " broken seek state: output code=" diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java new file mode 100644 index 00000000000..8c742bdd02a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java @@ -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 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); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java index 4ea4357c66a..36361b5d78e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Stats.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.blocktree; +package org.apache.lucene.codecs.lucene90.blocktree; import java.io.ByteArrayOutputStream; import java.io.PrintStream; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/package-info.java similarity index 60% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/package-info.java index c126827ef00..27d57b60371 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/package-info.java @@ -16,7 +16,14 @@ */ /** - * Components from the Lucene 8.4 index format. See {@link org.apache.lucene.codecs.lucene90} for an - * overview of the current index format. + * BlockTree terms dictionary. + * + *

    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. + * + *

    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; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/gen_ForUtil.py new file mode 100644 index 00000000000..30256182122 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/gen_ForUtil.py @@ -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') diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java index 4d34a40569d..b615cf7ef7f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/package-info.java @@ -151,15 +151,15 @@ * field names. These are used to store auxiliary information about the document, such as its * title, url, or an identifier to access a database. The set of stored fields are what is * returned for each hit when searching. This is keyed by document number. - *

  • {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term dictionary}. A + *
  • {@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 * documents. The dictionary also contains the number of documents which contain the term, and * pointers to the term's frequency and proximity data. - *
  • {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Frequency data}. For + *
  • {@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 * the frequency of the term in that document, unless frequencies are omitted ({@link * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) - *
  • {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Proximity data}. For + *
  • {@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 * this will not exist if all fields in all documents omit position data. *
  • {@link org.apache.lucene.codecs.lucene80.Lucene80NormsFormat Normalization factors}. For @@ -255,27 +255,27 @@ * The stored fields for documents * * - * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Dictionary} + * {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Dictionary} * .tim * The term dictionary, stores term info * * - * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Index} + * {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Term Index} * .tip * The index into the Term Dictionary * * - * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Frequencies} + * {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Frequencies} * .doc * Contains the list of docs which contain each term along with frequency * * - * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Positions} + * {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Positions} * .pos * Stores position information about where a term occurs in the index * * - * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Payloads} + * {@link org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat Payloads} * .pay * Stores additional per-position metadata information such as character offsets and user payloads * diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java index 3b9d327b7ec..26d35770661 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java @@ -21,8 +21,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.IndexReader; 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 * document for the positions of a term. See also {@link - * Lucene84PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. + * Lucene90PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. * *

    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), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -415,7 +415,7 @@ public class PhraseQuery extends Query { /** * 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. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 486b1057566..15681e8a2e5 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat +org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForDeltaUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForDeltaUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForDeltaUtil.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForDeltaUtil.java index 81408da162c..958d4b74967 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForDeltaUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForUtil.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForUtil.java index 171e8ad3998..b1f683ba005 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90PostingsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90PostingsFormat.java new file mode 100644 index 00000000000..df4aacc1b27 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90PostingsFormat.java @@ -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 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 impacts2 = + Lucene90ScoreSkipReader.readImpacts(new ByteArrayDataInput(b), new MutableImpactList()); + assertEquals(impacts, impacts2); + } + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestPForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestPForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestPForUtil.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestPForUtil.java index 0b3e1c146ef..b031700cc73 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene84/TestPForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestPForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene84; +package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java index 3b0c5770b5e..414d360d5d1 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Objects; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; 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 * document for the positions of a term. See also {@link - * Lucene84PostingsReader.EverythingEnum#nextPosition()}. + * Lucene90PostingsReader.EverythingEnum#nextPosition()}. * *

    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), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * 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; /** - * 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. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/IDVersionPostingsFormat.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/IDVersionPostingsFormat.java index 614e172e75b..aea044223f7 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/IDVersionPostingsFormat.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/IDVersionPostingsFormat.java @@ -22,7 +22,7 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.search.LiveFieldValues; @@ -64,14 +64,16 @@ public class IDVersionPostingsFormat extends PostingsFormat { private final int maxTermsInBlock; 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) { super("IDVersion"); this.minTermsInBlock = minTermsInBlock; this.maxTermsInBlock = maxTermsInBlock; - BlockTreeTermsWriter.validateSettings(minTermsInBlock, maxTermsInBlock); + Lucene90BlockTreeTermsWriter.validateSettings(minTermsInBlock, maxTermsInBlock); } @Override diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/VersionBlockTreeTermsWriter.java index 0f959a4e31d..fd2d2a648d8 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/VersionBlockTreeTermsWriter.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/codecs/idversion/VersionBlockTreeTermsWriter.java @@ -24,7 +24,7 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.NormsProducer; 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.FieldInfos; 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 - * adds a method to its TermsEnum implementation to seekExact only if the version is >= the + * This is just like {@link Lucene90BlockTreeTermsWriter}, except it also stores a version per term, + * 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 * block has a high enough version. The term blocks file is .tiv and the terms index extension is * .tipv. @@ -181,7 +181,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer { int minItemsInBlock, int maxItemsInBlock) throws IOException { - BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); + Lucene90BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); maxDoc = state.segmentInfo.maxDoc(); final String termsFileName = diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java index 03171174a7f..02536cc7f26 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java @@ -20,9 +20,9 @@ import org.apache.lucene.codecs.PostingsFormat; /** * {@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 - * Completion84PostingsFormat} on new indices. + * Completion90PostingsFormat} on new indices. * * @lucene.experimental */ diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion84PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion84PostingsFormat.java index 2057a2bf4e9..8551fc3a80a 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion84PostingsFormat.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion84PostingsFormat.java @@ -19,8 +19,10 @@ package org.apache.lucene.search.suggest.document; import org.apache.lucene.codecs.PostingsFormat; /** - * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@link - * org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat} + * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code + * 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 */ diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion90PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion90PostingsFormat.java new file mode 100644 index 00000000000..cf2a9103aa7 --- /dev/null +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion90PostingsFormat.java @@ -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 fstLoadMode + * 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"); + } +} diff --git a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index b74211eb8dd..a4070471047 100644 --- a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -32,3 +32,4 @@ org.apache.lucene.search.suggest.document.Completion50PostingsFormat org.apache.lucene.search.suggest.document.Completion84PostingsFormat +org.apache.lucene.search.suggest.document.Completion90PostingsFormat diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java index 7ebac3f6cec..36d16c3d255 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java @@ -964,7 +964,7 @@ public class TestSuggestField extends LuceneTestCase { new Lucene90Codec() { CompletionPostingsFormat.FSTLoadMode fstLoadMode = RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); - PostingsFormat postingsFormat = new Completion84PostingsFormat(fstLoadMode); + PostingsFormat postingsFormat = new Completion90PostingsFormat(fstLoadMode); @Override public PostingsFormat getPostingsFormatForField(String field) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneFixedGap.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneFixedGap.java index fadfaf2f28c..0d9f1e05832 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneFixedGap.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneFixedGap.java @@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState; // 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 { final int termIndexInterval; @@ -48,7 +48,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override 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 // pluggable? Ie so that this codec would record which @@ -85,7 +85,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene84PostingsReader(state); + PostingsReaderBase postings = new Lucene90PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapDocFreqInterval.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapDocFreqInterval.java index ce17f386d93..765885a46b7 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapDocFreqInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapDocFreqInterval.java @@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState; // 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 * terms. */ @@ -52,7 +52,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override 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 // pluggable? Ie so that this codec would record which @@ -93,7 +93,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene84PostingsReader(state); + PostingsReaderBase postings = new Lucene90PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapFixedInterval.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapFixedInterval.java index 44c050f059a..7fc9201b0d3 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapFixedInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/blockterms/LuceneVarGapFixedInterval.java @@ -22,9 +22,9 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat; // javadocs +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -32,7 +32,7 @@ import org.apache.lucene.index.SegmentWriteState; // 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. */ public final class LuceneVarGapFixedInterval extends PostingsFormat { @@ -49,7 +49,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override 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 // pluggable? Ie so that this codec would record which @@ -88,7 +88,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene84PostingsReader(state); + PostingsReaderBase postings = new Lucene90PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java index b26beba6fd2..c0914b3740a 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java @@ -33,12 +33,12 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; 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.OrdsBlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +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.FSTTermsWriter; import org.apache.lucene.index.FieldInfo; @@ -121,7 +121,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { random.nextInt(); // consume a random for buffersize - PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(state); final FieldsConsumer fields; final int t1 = random.nextInt(4); @@ -150,7 +150,9 @@ public final class MockRandomPostingsFormat extends PostingsFormat { boolean success = false; try { - fields = new BlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock); + fields = + new Lucene90BlockTreeTermsWriter( + state, postingsWriter, minTermsInBlock, maxTermsInBlock); success = true; } finally { if (!success) { @@ -287,7 +289,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); } - PostingsReaderBase postingsReader = new Lucene84PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene90PostingsReader(state); final FieldsProducer fields; final int t1 = random.nextInt(4); @@ -309,7 +311,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { boolean success = false; try { - fields = new BlockTreeTermsReader(postingsReader, state); + fields = new Lucene90BlockTreeTermsReader(postingsReader, state); success = true; } finally { if (!success) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java index 54ab0fa91e6..ef082f22277 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsReader; -import org.apache.lucene.codecs.lucene84.Lucene84PostingsWriter; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsReader; +import org.apache.lucene.codecs.lucene90.Lucene90PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.store.ByteBuffersDataOutput; @@ -61,7 +61,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException { - PostingsWriterBase postingsWriter = new Lucene84PostingsWriter(segmentWriteState); + PostingsWriterBase postingsWriter = new Lucene90PostingsWriter(segmentWriteState); boolean success = false; try { FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter); @@ -139,7 +139,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException { - PostingsReaderBase postingsReader = new Lucene84PostingsReader(segmentReadState); + PostingsReaderBase postingsReader = new Lucene90PostingsReader(segmentReadState); boolean success = false; try { FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader); diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java index 1321a43f880..d6615d5c9a3 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java @@ -54,8 +54,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec; import org.apache.lucene.codecs.blockterms.LuceneFixedGap; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; 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.Lucene90PostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.document.BinaryDocValuesField; @@ -1219,7 +1219,7 @@ public final class TestUtil { * Lucene. */ public static PostingsFormat getDefaultPostingsFormat() { - return new Lucene84PostingsFormat(); + return new Lucene90PostingsFormat(); } /** @@ -1230,7 +1230,7 @@ public final class TestUtil { */ public static PostingsFormat getDefaultPostingsFormat( int minItemsPerBlock, int maxItemsPerBlock) { - return new Lucene84PostingsFormat(minItemsPerBlock, maxItemsPerBlock); + return new Lucene90PostingsFormat(minItemsPerBlock, maxItemsPerBlock); } /** Returns a random postings format that supports term ordinals */ diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml index e2add4eb2a8..b8a1a0b5e56 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml @@ -20,7 +20,7 @@ as a way to vet that the configuration actually matters. --> - +