diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index cedb02f2237..ea184b7cea9 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -115,6 +115,10 @@ New Features minimum numeric values from the provided Terms. (Robert Muir, Mike McCandless) +* LUCENE-5675: Add IDVersionPostingsFormat, a postings format + optimized for primary-key (ID) fields that also record a version + (long) for each ID. (Robert Muir, Mike McCandless) + Changes in Backwards Compatibility Policy * LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java index 9946062e09b..6bb0466608b 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java @@ -17,7 +17,7 @@ package org.apache.lucene.codecs.pulsing; * limitations under the License. */ -import org.apache.lucene.codecs.BlockTreeTermsWriter; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat; import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java index 551743f1aa8..6bc4451b576 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java @@ -19,14 +19,14 @@ package org.apache.lucene.codecs.pulsing; import java.io.IOException; -import org.apache.lucene.codecs.BlockTreeTermsReader; -import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsBaseFormat; 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.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java deleted file mode 100644 index 54e37e5a05b..00000000000 --- a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java +++ /dev/null @@ -1,3042 +0,0 @@ -package org.apache.lucene.codecs; - -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.util.Collections; -import java.util.Iterator; -import java.util.Locale; -import java.util.TreeMap; - -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.index.DocsEnum; -import org.apache.lucene.index.FieldInfo.IndexOptions; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.TermState; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; -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.util.ArrayUtil; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.StringHelper; -import org.apache.lucene.util.automaton.CompiledAutomaton; -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; -import org.apache.lucene.util.fst.Util; - -/** 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 it's 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 BlockTreeTermsWriter}. - * - * @lucene.experimental - */ - -public class BlockTreeTermsReader extends FieldsProducer { - - // Open input to the main terms dict file (_X.tib) - private final IndexInput in; - - //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG; - - // Reads the terms dict entries, to gather state to - // produce DocsEnum on demand - private final PostingsReaderBase postingsReader; - - private final TreeMap fields = new TreeMap<>(); - - /** File offset where the directory starts in the terms file. */ - private long dirOffset; - - /** File offset where the directory starts in the index file. */ - private long indexDirOffset; - - private String segment; - - private final int version; - - /** Sole constructor. */ - public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info, - PostingsReaderBase postingsReader, IOContext ioContext, - String segmentSuffix) - throws IOException { - - this.postingsReader = postingsReader; - - this.segment = info.name; - in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION), - ioContext); - - boolean success = false; - IndexInput indexIn = null; - - try { - version = readHeader(in); - indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION), - ioContext); - int indexVersion = readIndexHeader(indexIn); - if (indexVersion != version) { - throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion); - } - - // verify - if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) { - CodecUtil.checksumEntireFile(indexIn); - } - - // Have PostingsReader init itself - postingsReader.init(in); - - // Read per-field details - seekDir(in, dirOffset); - seekDir(indexIn, indexDirOffset); - - final int numFields = in.readVInt(); - if (numFields < 0) { - throw new CorruptIndexException("invalid numFields: " + numFields + " (resource=" + in + ")"); - } - - for(int i=0;i= 0; - final int numBytes = in.readVInt(); - final BytesRef rootCode = new BytesRef(new byte[numBytes]); - in.readBytes(rootCode.bytes, 0, numBytes); - rootCode.length = numBytes; - final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); - assert fieldInfo != null: "field=" + field; - final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong(); - final long sumDocFreq = in.readVLong(); - final int docCount = in.readVInt(); - final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0; - - BytesRef minTerm, maxTerm; - if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) { - minTerm = readBytesRef(in); - maxTerm = readBytesRef(in); - } else { - minTerm = maxTerm = null; - } - if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs - throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")"); - } - if (sumDocFreq < docCount) { // #postings must be >= #docs with field - throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")"); - } - if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings - throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")"); - } - final long indexStartFP = indexIn.readVLong(); - FieldReader previous = fields.put(fieldInfo.name, - new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, - indexStartFP, longsSize, indexIn, minTerm, maxTerm)); - if (previous != null) { - throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")"); - } - } - indexIn.close(); - - success = true; - } finally { - if (!success) { - // this.close() will close in: - IOUtils.closeWhileHandlingException(indexIn, this); - } - } - } - - private static BytesRef readBytesRef(IndexInput in) throws IOException { - BytesRef bytes = new BytesRef(); - bytes.length = in.readVInt(); - bytes.bytes = new byte[bytes.length]; - in.readBytes(bytes.bytes, 0, bytes.length); - return bytes; - } - - /** Reads terms file header. */ - private int readHeader(IndexInput input) throws IOException { - int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME, - BlockTreeTermsWriter.VERSION_START, - BlockTreeTermsWriter.VERSION_CURRENT); - if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) { - dirOffset = input.readLong(); - } - return version; - } - - /** Reads index file header. */ - private int readIndexHeader(IndexInput input) throws IOException { - int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME, - BlockTreeTermsWriter.VERSION_START, - BlockTreeTermsWriter.VERSION_CURRENT); - if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) { - indexDirOffset = input.readLong(); - } - return version; - } - - /** Seek {@code input} to the directory offset. */ - private void seekDir(IndexInput input, long dirOffset) - throws IOException { - if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) { - input.seek(input.length() - CodecUtil.footerLength() - 8); - dirOffset = input.readLong(); - } else if (version >= BlockTreeTermsWriter.VERSION_APPEND_ONLY) { - input.seek(input.length() - 8); - dirOffset = input.readLong(); - } - input.seek(dirOffset); - } - - // for debugging - // private static String toHex(int v) { - // return "0x" + Integer.toHexString(v); - // } - - @Override - public void close() throws IOException { - try { - IOUtils.close(in, postingsReader); - } finally { - // Clear so refs to terms index is GCable even if - // app hangs onto us: - fields.clear(); - } - } - - @Override - public Iterator iterator() { - return Collections.unmodifiableSet(fields.keySet()).iterator(); - } - - @Override - public Terms terms(String field) throws IOException { - assert field != null; - return fields.get(field); - } - - @Override - public int size() { - return fields.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(); - } - } - } - - /** - * BlockTree statistics for a single field - * returned by {@link FieldReader#computeStats()}. - */ - public static class Stats { - /** How many nodes in the index FST. */ - public long indexNodeCount; - - /** How many arcs in the index FST. */ - public long indexArcCount; - - /** 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; - - /** Total number of bytes used to store term stats (not - * including what the {@link PostingsBaseFormat} - * stores. */ - public long totalBlockStatsBytes; - - /** Total bytes stored by the {@link PostingsBaseFormat}, - * 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(FieldReader.SegmentTermsEnum.Frame 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.suffixesReader.length(); - totalBlockStatsBytes += frame.statsReader.length(); - } - - void endBlock(FieldReader.SegmentTermsEnum.Frame 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.suffixesReader.length() - 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(" " + indexNodeCount + " nodes"); - out.println(" " + indexArcCount + " arcs"); - 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(" " + totalBlockSuffixBytes + " term suffix bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : "")); - 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 fstOutputs = ByteSequenceOutputs.getSingleton(); - final BytesRef NO_OUTPUT = fstOutputs.getNoOutput(); - - /** BlockTree's implementation of {@link Terms}. */ - public final class FieldReader extends Terms { - final long numTerms; - final FieldInfo fieldInfo; - final long sumTotalTermFreq; - final long sumDocFreq; - final int docCount; - final long indexStartFP; - final long rootBlockFP; - final BytesRef rootCode; - final BytesRef minTerm; - final BytesRef maxTerm; - final int longsSize; - - private final FST index; - //private boolean DEBUG; - - FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, - long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException { - assert numTerms > 0; - this.fieldInfo = fieldInfo; - //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id"); - this.numTerms = numTerms; - this.sumTotalTermFreq = sumTotalTermFreq; - this.sumDocFreq = sumDocFreq; - this.docCount = docCount; - this.indexStartFP = indexStartFP; - this.rootCode = rootCode; - this.longsSize = longsSize; - 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() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; - - if (indexIn != null) { - final IndexInput clone = indexIn.clone(); - //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name); - clone.seek(indexStartFP); - index = new FST<>(clone, ByteSequenceOutputs.getSingleton()); - - /* - 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(); - } - */ - } else { - index = null; - } - } - - @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*/ - // TODO: maybe push this into Terms? - public Stats computeStats() throws IOException { - return new SegmentTermsEnum().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(TermsEnum reuse) throws IOException { - return new SegmentTermsEnum(); - } - - @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 (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) { - throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead"); - } - return new IntersectEnum(compiled, startTerm); - } - - /** Returns approximate RAM bytes used */ - public long ramBytesUsed() { - return ((index!=null)? index.sizeInBytes() : 0); - } - - // NOTE: cannot seek! - private final class IntersectEnum extends TermsEnum { - private final IndexInput in; - - private Frame[] stack; - - @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc[] arcs = new FST.Arc[5]; - - private final RunAutomaton runAutomaton; - private final CompiledAutomaton compiledAutomaton; - - private Frame currentFrame; - - private final BytesRef term = new BytesRef(); - - private final FST.BytesReader fstReader; - - // TODO: can we share this with the frame in STE? - private final class Frame { - final int ord; - long fp; - long fpOrig; - long fpEnd; - long lastSubFP; - - // State in automaton - int state; - - int metaDataUpto; - - byte[] suffixBytes = new byte[128]; - final ByteArrayDataInput suffixesReader = new ByteArrayDataInput(); - - byte[] statBytes = new byte[64]; - 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; - - Transition[] transitions; - int curTransitionMax; - int transitionIndex; - - FST.Arc arc; - - final BlockTermState termState; - - // metadata buffer, holding monotonic values - public long[] longs; - // metadata buffer, holding general values - public byte[] bytes; - ByteArrayDataInput bytesReader; - - // Cumulative output so far - BytesRef outputPrefix; - - private int startBytePos; - private int suffix; - - public Frame(int ord) throws IOException { - this.ord = ord; - this.termState = postingsReader.newTermState(); - this.termState.totalTermFreq = -1; - this.longs = new long[longsSize]; - } - - void loadNextFloorBlock() throws IOException { - assert numFollowFloorBlocks > 0; - //if (DEBUG) System.out.println(" loadNextFoorBlock trans=" + transitions[transitionIndex]); - - do { - fp = fpOrig + (floorDataReader.readVLong() >>> 1); - numFollowFloorBlocks--; - // if (DEBUG) System.out.println(" skip floor block2! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks); - if (numFollowFloorBlocks != 0) { - nextFloorLabel = floorDataReader.readByte() & 0xff; - } else { - nextFloorLabel = 256; - } - // if (DEBUG) System.out.println(" nextFloorLabel=" + (char) nextFloorLabel); - } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin()); - - load(null); - } - - public void setState(int state) { - this.state = state; - transitionIndex = 0; - transitions = compiledAutomaton.sortedTransitions[state]; - if (transitions.length != 0) { - curTransitionMax = transitions[0].getMax(); - } else { - curTransitionMax = -1; - } - } - - void load(BytesRef frameIndexData) throws IOException { - - // if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state)); - - if (frameIndexData != null && transitions.length != 0) { - // Floor frame - if (floorData.length < frameIndexData.length) { - this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)]; - } - System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length); - floorDataReader.reset(floorData, 0, frameIndexData.length); - // Skip first long -- has redundant fp, hasTerms - // flag, isFloor flag - final long code = floorDataReader.readVLong(); - if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) { - numFollowFloorBlocks = floorDataReader.readVInt(); - nextFloorLabel = floorDataReader.readByte() & 0xff; - // if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel); - - // If current state is accept, we must process - // first block in case it has empty suffix: - if (!runAutomaton.isAccept(state)) { - // Maybe skip floor blocks: - while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) { - fp = fpOrig + (floorDataReader.readVLong() >>> 1); - numFollowFloorBlocks--; - // if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks); - if (numFollowFloorBlocks != 0) { - nextFloorLabel = floorDataReader.readByte() & 0xff; - } else { - nextFloorLabel = 256; - } - } - } - } - } - - in.seek(fp); - int code = in.readVInt(); - entCount = code >>> 1; - assert entCount > 0; - isLastInFloor = (code & 1) != 0; - - // term suffixes: - code = in.readVInt(); - isLeafBlock = (code & 1) != 0; - int numBytes = code >>> 1; - // if (DEBUG) System.out.println(" entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes); - if (suffixBytes.length < numBytes) { - suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - in.readBytes(suffixBytes, 0, numBytes); - suffixesReader.reset(suffixBytes, 0, numBytes); - - // stats - numBytes = in.readVInt(); - if (statBytes.length < numBytes) { - statBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - in.readBytes(statBytes, 0, numBytes); - statsReader.reset(statBytes, 0, numBytes); - metaDataUpto = 0; - - termState.termBlockOrd = 0; - nextEnt = 0; - - // metadata - numBytes = in.readVInt(); - if (bytes == null) { - bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - bytesReader = new ByteArrayDataInput(); - } else if (bytes.length < numBytes) { - bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - 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 = in.getFilePointer(); - } - } - - // TODO: maybe add scanToLabel; should give perf boost - - public boolean next() { - return isLeafBlock ? nextLeaf() : nextNonLeaf(); - } - - // Decodes next entry; returns true if it's a sub-block - public boolean 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 = suffixesReader.readVInt(); - startBytePos = suffixesReader.getPosition(); - suffixesReader.skipBytes(suffix); - return false; - } - - public boolean nextNonLeaf() { - //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++; - final int code = suffixesReader.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 - suffixesReader.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 - termState.docFreq = statsReader.readVInt(); - //if (DEBUG) System.out.println(" dF=" + state.docFreq); - if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) { - termState.totalTermFreq = termState.docFreq + statsReader.readVLong(); - //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); - } - // metadata - for (int i = 0; i < longsSize; i++) { - longs[i] = bytesReader.readVLong(); - } - postingsReader.decodeTerm(longs, bytesReader, fieldInfo, termState, absolute); - - metaDataUpto++; - absolute = false; - } - termState.termBlockOrd = metaDataUpto; - } - } - - private BytesRef savedStartTerm; - - // TODO: in some cases we can filter by length? eg - // regexp foo*bar must be at least length 6 bytes - public IntersectEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException { - // if (DEBUG) { - // System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef)); - // } - runAutomaton = compiled.runAutomaton; - compiledAutomaton = compiled; - in = BlockTreeTermsReader.this.in.clone(); - stack = new Frame[5]; - for(int idx=0;idx(); - } - - if (index == null) { - fstReader = null; - } else { - fstReader = 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 = 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 Frame f = stack[0]; - f.fp = f.fpOrig = rootBlockFP; - f.prefix = 0; - f.setState(runAutomaton.getInitialState()); - f.arc = arc; - f.outputPrefix = arc.output; - f.load(rootCode); - - // for assert: - assert setSavedStartTerm(startTerm); - - currentFrame = f; - if (startTerm != null) { - seekToStartTerm(startTerm); - } - } - - // 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 Frame getFrame(int ord) throws IOException { - if (ord >= stack.length) { - final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(stack, 0, next, 0, stack.length); - for(int stackOrd=stack.length;stackOrd 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(); - } - arcs = next; - } - return arcs[ord]; - } - - private Frame pushFrame(int state) throws IOException { - final Frame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord); - - f.fp = f.fpOrig = currentFrame.lastSubFP; - f.prefix = currentFrame.prefix + currentFrame.suffix; - // if (DEBUG) System.out.println(" pushFrame state=" + state + " prefix=" + f.prefix); - 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 = 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 { - //if (DEBUG) System.out.println("BTIR.docFreq"); - currentFrame.decodeMetaData(); - //if (DEBUG) System.out.println(" return " + currentFrame.termState.docFreq); - return currentFrame.termState.docFreq; - } - - @Override - public long totalTermFreq() throws IOException { - currentFrame.decodeMetaData(); - return currentFrame.termState.totalTermFreq; - } - - @Override - public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException { - currentFrame.decodeMetaData(); - return postingsReader.docs(fieldInfo, currentFrame.termState, skipDocs, reuse, flags); - } - - @Override - public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { - if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { - // Positions were not indexed: - return null; - } - - currentFrame.decodeMetaData(); - return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse, flags); - } - - private int getState() { - int state = currentFrame.state; - for(int idx=0;idx 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++]) { - // if (DEBUG) { - // System.out.println(" skip: common suffix mismatch (in prefix)"); - // } - continue nextTerm; - } - } - suffixBytesPos = currentFrame.startBytePos; - } else { - suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length; - } - - // Test overlapping suffix part: - final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length; - while (commonSuffixBytesPos < commonSuffixBytesPosEnd) { - if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) { - // if (DEBUG) { - // System.out.println(" skip: common suffix mismatch"); - // } - 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 prefix matches the automaton: - int state = currentFrame.state; - for (int idx=0;idx[] arcs = - new FST.Arc[1]; - - public SegmentTermsEnum() throws IOException { - //if (DEBUG) System.out.println("BTTR.init seg=" + segment); - stack = new Frame[0]; - - // Used to hold seek by TermState, or cached seek - staticFrame = new Frame(-1); - - if (index == null) { - fstReader = null; - } else { - fstReader = 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(); - } - - currentFrame = staticFrame; - final FST.Arc arc; - if (index != null) { - arc = index.getFirstArc(arcs[0]); - // Empty string prefix must have an output in the index! - assert arc.isFinal(); - } else { - arc = null; - } - currentFrame = staticFrame; - //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 = BlockTreeTermsReader.this.in.clone(); - } - } - - /** Runs next() through the entire terms dict, - * computing aggregate statistics. */ - public Stats computeBlockStats() throws IOException { - - Stats stats = new Stats(segment, fieldInfo.name); - if (index != null) { - stats.indexNodeCount = index.getNodeCount(); - stats.indexArcCount = index.getArcCount(); - stats.indexNumBytes = index.sizeInBytes(); - } - - currentFrame = staticFrame; - FST.Arc arc; - if (index != null) { - arc = 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, 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) { - currentFrame.loadNextFloorBlock(); - stats.startBlock(currentFrame, true); - } 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.isFloor = false; - //currentFrame.hasTerms = true; - currentFrame.loadBlock(); - stats.startBlock(currentFrame, !currentFrame.isLastInFloor); - } else { - stats.term(term); - break; - } - } - } - - stats.finish(); - - // Put root frame back: - currentFrame = staticFrame; - if (index != null) { - arc = 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.rewind(); - currentFrame.loadBlock(); - validIndexPrefix = 0; - term.length = 0; - - return stats; - } - - private Frame getFrame(int ord) throws IOException { - if (ord >= stack.length) { - final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(stack, 0, next, 0, stack.length); - for(int stackOrd=stack.length;stackOrd 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(); - } - arcs = next; - } - return arcs[ord]; - } - - // Pushes a frame we seek'd to - Frame 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 >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; - final Frame f = getFrame(1+currentFrame.ord); - f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0; - f.hasTermsOrig = f.hasTerms; - f.isFloor = (code & BlockTreeTermsWriter.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 - Frame pushFrame(FST.Arc arc, long fp, int length) throws IOException { - final Frame 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) { - 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; - } - - @Override - public boolean seekExact(final BytesRef target) throws IOException { - - if (index == null) { - throw new IllegalStateException("terms index was not loaded"); - } - - if (term.bytes.length <= target.length) { - term.bytes = ArrayUtil.grow(term.bytes, 1+target.length); - } - - assert clearEOF(); - - // if (DEBUG) { - // System.out.println("\nBTTR.seekExact seg=" + segment + " target=" + fieldInfo.name + ":" + brToString(target) + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=" + validIndexPrefix); - // printSeekState(); - // } - - 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; - - Frame 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.bytes[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]; - //if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) { - //System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF)); - //} - 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 != NO_OUTPUT) { - output = fstOutputs.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.bytes[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 = 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 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 = 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, fstOutputs.add(output, arc.nextFinalOutput), 0); - } - - // if (DEBUG) { - // System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength); - // } - - while (targetUpto < target.length) { - - final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF; - - final FST.Arc nextArc = 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.bytes[targetUpto] = (byte) targetLabel; - term.length = 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.bytes[targetUpto] = (byte) targetLabel; - // Aggregate output as we go: - assert arc.output != null; - if (arc.output != NO_OUTPUT) { - output = fstOutputs.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, fstOutputs.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.length = 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(final BytesRef target) throws IOException { - if (index == null) { - throw new IllegalStateException("terms index was not loaded"); - } - - if (term.bytes.length <= target.length) { - term.bytes = ArrayUtil.grow(term.bytes, 1+target.length); - } - - assert clearEOF(); - - //if (DEBUG) { - //System.out.println("\nBTTR.seekCeil seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix= " + validIndexPrefix); - //printSeekState(); - //} - - 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; - - Frame lastFrame = stack[0]; - assert validIndexPrefix <= term.length; - - final int targetLimit = Math.min(target.length, validIndexPrefix); - - int cmp = 0; - - // TOOD: 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.bytes[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); - // TOOD: 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 != NO_OUTPUT) { - output = fstOutputs.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.bytes[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 + "); 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 = 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, fstOutputs.add(output, arc.nextFinalOutput), 0); - } - - //if (DEBUG) { - //System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord+1=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength); - //} - - while (targetUpto < target.length) { - - final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF; - - final FST.Arc nextArc = 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); - - 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=" + brToString(term) + " " + 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) + " " + term); - //} - return result; - } - } else { - // Follow this arc - term.bytes[targetUpto] = (byte) targetLabel; - arc = nextArc; - // Aggregate output as we go: - assert arc.output != null; - if (arc.output != NO_OUTPUT) { - output = fstOutputs.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, fstOutputs.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.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) { - Frame f = getFrame(ord); - assert f != null; - final BytesRef prefix = new BytesRef(term.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< arc; - if (index != null) { - arc = 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(); - } - - targetBeforeCurrentLength = currentFrame.ord; - - assert !eof; - //if (DEBUG) { - //System.out.println("\nBTTR.next seg=" + segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix); - //printSeekState(); - //} - - 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); - assert result; - } - - // Pop finished blocks - while (currentFrame.nextEnt == currentFrame.entCount) { - if (!currentFrame.isLastInFloor) { - currentFrame.loadNextFloorBlock(); - } else { - //if (DEBUG) System.out.println(" pop frame"); - if (currentFrame.ord == 0) { - //if (DEBUG) System.out.println(" return null"); - assert setEOF(); - term.length = 0; - 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); - 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.isFloor = false; - //currentFrame.hasTerms = true; - currentFrame.loadBlock(); - } else { - //if (DEBUG) System.out.println(" return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord); - return term; - } - } - } - - @Override - public BytesRef term() { - assert !eof; - return term; - } - - @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 DocsEnum docs(Bits skipDocs, DocsEnum 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 postingsReader.docs(fieldInfo, currentFrame.state, skipDocs, reuse, flags); - } - - @Override - public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { - if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { - // Positions were not indexed: - return null; - } - - assert !eof; - currentFrame.decodeMetaData(); - return postingsReader.docsAndPositions(fieldInfo, currentFrame.state, skipDocs, reuse, 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) != 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(); - } - - // Not static -- references term, postingsReader, - // fieldInfo, in - private final class Frame { - // Our index in stack[]: - final int ord; - - boolean hasTerms; - boolean hasTermsOrig; - boolean isFloor; - - FST.Arc arc; - - // File pointer where this block was loaded from - long fp; - long fpOrig; - long fpEnd; - - byte[] suffixBytes = new byte[128]; - final ByteArrayDataInput suffixesReader = new ByteArrayDataInput(); - - byte[] statBytes = new byte[64]; - 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, holding monotonic values - public long[] longs; - // metadata buffer, holding general values - public byte[] bytes; - ByteArrayDataInput bytesReader; - - public Frame(int ord) throws IOException { - this.ord = ord; - this.state = postingsReader.newTermState(); - this.state.totalTermFreq = -1; - this.longs = new long[longsSize]; - } - - 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: - initIndexInput(); - - if (nextEnt != -1) { - // Already loaded - return; - } - //System.out.println("blc=" + blockLoadCount); - - in.seek(fp); - int code = in.readVInt(); - entCount = code >>> 1; - assert entCount > 0; - isLastInFloor = (code & 1) != 0; - assert arc == null || (isLastInFloor || isFloor); - - // 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 - - // term suffixes: - code = in.readVInt(); - isLeafBlock = (code & 1) != 0; - int numBytes = code >>> 1; - if (suffixBytes.length < numBytes) { - suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - in.readBytes(suffixBytes, 0, numBytes); - suffixesReader.reset(suffixBytes, 0, numBytes); - - /*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 - numBytes = in.readVInt(); - if (statBytes.length < numBytes) { - statBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - in.readBytes(statBytes, 0, numBytes); - statsReader.reset(statBytes, 0, numBytes); - 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 = in.readVInt(); - if (bytes == null) { - bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - bytesReader = new ByteArrayDataInput(); - } else if (bytes.length < numBytes) { - bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; - } - 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 = 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(); - 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); - } - */ - } - - public boolean next() { - return isLeafBlock ? nextLeaf() : nextNonLeaf(); - } - - // Decodes next entry; returns true if it's a sub-block - public boolean 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 = suffixesReader.readVInt(); - startBytePos = suffixesReader.getPosition(); - term.length = prefix + suffix; - if (term.bytes.length < term.length) { - term.grow(term.length); - } - suffixesReader.readBytes(term.bytes, prefix, suffix); - // A normal term - termExists = true; - return false; - } - - public boolean nextNonLeaf() { - //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++; - final int code = suffixesReader.readVInt(); - suffix = code >>> 1; - startBytePos = suffixesReader.getPosition(); - term.length = prefix + suffix; - if (term.bytes.length < term.length) { - term.grow(term.length); - } - suffixesReader.readBytes(term.bytes, prefix, suffix); - if ((code & 1) == 0) { - // A normal term - termExists = true; - subCode = 0; - state.termBlockOrd++; - return false; - } else { - // A sub-block; make sub-FP absolute: - termExists = false; - subCode = suffixesReader.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: - - // stats - state.docFreq = statsReader.readVInt(); - //if (DEBUG) System.out.println(" dF=" + state.docFreq); - if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) { - state.totalTermFreq = state.docFreq + statsReader.readVLong(); - //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); - } - // metadata - for (int i = 0; i < longsSize; i++) { - longs[i] = bytesReader.readVLong(); - } - postingsReader.decodeTerm(longs, bytesReader, fieldInfo, state, absolute); - - metaDataUpto++; - absolute = false; - } - state.termBlockOrd = metaDataUpto; - } - - // Used only by assert - private boolean prefixMatches(BytesRef target) { - for(int bytePos=0;bytePos= BlockTreeTermsWriter.VERSION_CHECKSUM) { - // term dictionary - CodecUtil.checksumEntireFile(in); - - // postings - postingsReader.checkIntegrity(); - } - } -} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java index 39476ed3f65..8cd194f1141 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java @@ -37,6 +37,8 @@ import org.apache.lucene.util.Bits; * time. * @lucene.experimental */ +// TODO: maybe move under blocktree? but it's used by other terms dicts (e.g. Block) + // TODO: find a better name; this defines the API that the // terms dict impls use to talk to a postings impl. // TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer 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 645ee0184b4..6e083164bec 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java @@ -20,6 +20,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.index.DocsAndPositionsEnum; // javadocs import org.apache.lucene.index.DocsEnum; // javadocs import org.apache.lucene.index.FieldInfo; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java new file mode 100644 index 00000000000..5ef41bc4596 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java @@ -0,0 +1,313 @@ +package org.apache.lucene.codecs.blocktree; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.io.PrintStream; +import java.util.Collections; +import java.util.Iterator; +import java.util.TreeMap; + +import org.apache.lucene.codecs.BlockTermState; +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.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +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.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.automaton.CompiledAutomaton; +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; +import org.apache.lucene.util.fst.Util; + +/** 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 it's 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 BlockTreeTermsWriter}. + * + * @lucene.experimental + */ + +public final class BlockTreeTermsReader extends FieldsProducer { + + // Open input to the main terms dict file (_X.tib) + final IndexInput in; + + //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 TreeMap fields = new TreeMap<>(); + + /** File offset where the directory starts in the terms file. */ + private long dirOffset; + + /** File offset where the directory starts in the index file. */ + private long indexDirOffset; + + final String segment; + + private final int version; + + /** Sole constructor. */ + public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info, + PostingsReaderBase postingsReader, IOContext ioContext, + String segmentSuffix) + throws IOException { + + this.postingsReader = postingsReader; + + this.segment = info.name; + in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION), + ioContext); + + boolean success = false; + IndexInput indexIn = null; + + try { + version = readHeader(in); + indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION), + ioContext); + int indexVersion = readIndexHeader(indexIn); + if (indexVersion != version) { + throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion); + } + + // verify + if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) { + CodecUtil.checksumEntireFile(indexIn); + } + + // Have PostingsReader init itself + postingsReader.init(in); + + // Read per-field details + seekDir(in, dirOffset); + seekDir(indexIn, indexDirOffset); + + final int numFields = in.readVInt(); + if (numFields < 0) { + throw new CorruptIndexException("invalid numFields: " + numFields + " (resource=" + in + ")"); + } + + for(int i=0;i= 0; + final int numBytes = in.readVInt(); + final BytesRef rootCode = new BytesRef(new byte[numBytes]); + in.readBytes(rootCode.bytes, 0, numBytes); + rootCode.length = numBytes; + final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + assert fieldInfo != null: "field=" + field; + final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong(); + final long sumDocFreq = in.readVLong(); + final int docCount = in.readVInt(); + final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0; + + BytesRef minTerm, maxTerm; + if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) { + minTerm = readBytesRef(in); + maxTerm = readBytesRef(in); + } else { + minTerm = maxTerm = null; + } + if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs + throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")"); + } + if (sumDocFreq < docCount) { // #postings must be >= #docs with field + throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")"); + } + if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings + throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")"); + } + final long indexStartFP = indexIn.readVLong(); + FieldReader previous = fields.put(fieldInfo.name, + new FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, + indexStartFP, longsSize, indexIn, minTerm, maxTerm)); + if (previous != null) { + throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")"); + } + } + indexIn.close(); + + success = true; + } finally { + if (!success) { + // this.close() will close in: + IOUtils.closeWhileHandlingException(indexIn, this); + } + } + } + + private static BytesRef readBytesRef(IndexInput in) throws IOException { + BytesRef bytes = new BytesRef(); + bytes.length = in.readVInt(); + bytes.bytes = new byte[bytes.length]; + in.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + + /** Reads terms file header. */ + private int readHeader(IndexInput input) throws IOException { + int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME, + BlockTreeTermsWriter.VERSION_START, + BlockTreeTermsWriter.VERSION_CURRENT); + if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) { + dirOffset = input.readLong(); + } + return version; + } + + /** Reads index file header. */ + private int readIndexHeader(IndexInput input) throws IOException { + int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME, + BlockTreeTermsWriter.VERSION_START, + BlockTreeTermsWriter.VERSION_CURRENT); + if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) { + indexDirOffset = input.readLong(); + } + return version; + } + + /** Seek {@code input} to the directory offset. */ + private void seekDir(IndexInput input, long dirOffset) + throws IOException { + if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) { + input.seek(input.length() - CodecUtil.footerLength() - 8); + dirOffset = input.readLong(); + } else if (version >= BlockTreeTermsWriter.VERSION_APPEND_ONLY) { + input.seek(input.length() - 8); + dirOffset = input.readLong(); + } + input.seek(dirOffset); + } + + // for debugging + // private static String toHex(int v) { + // return "0x" + Integer.toHexString(v); + // } + + @Override + public void close() throws IOException { + try { + IOUtils.close(in, postingsReader); + } finally { + // Clear so refs to terms index is GCable even if + // app hangs onto us: + fields.clear(); + } + } + + @Override + public Iterator iterator() { + return Collections.unmodifiableSet(fields.keySet()).iterator(); + } + + @Override + public Terms terms(String field) throws IOException { + assert field != null; + return fields.get(field); + } + + @Override + public int size() { + return fields.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 sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0); + for(FieldReader reader : fields.values()) { + sizeInByes += reader.ramBytesUsed(); + } + return sizeInByes; + } + + @Override + public void checkIntegrity() throws IOException { + if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) { + // term dictionary + CodecUtil.checksumEntireFile(in); + + // postings + postingsReader.checkIntegrity(); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java rename to lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java index ad08498d0d8..8948d82c509 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java @@ -1,4 +1,4 @@ -package org.apache.lucene.codecs; +package org.apache.lucene.codecs.blocktree; /* * Licensed to the Apache Software Foundation (ASF) under one or more @@ -21,6 +21,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; @@ -42,6 +46,7 @@ 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.NoOutputs; +import org.apache.lucene.util.fst.Outputs; import org.apache.lucene.util.fst.Util; import org.apache.lucene.util.packed.PackedInts; @@ -183,7 +188,11 @@ import org.apache.lucene.util.packed.PackedInts; * @see BlockTreeTermsReader * @lucene.experimental */ -public class BlockTreeTermsWriter extends FieldsConsumer { +public final class BlockTreeTermsWriter extends FieldsConsumer { + + static final Outputs FST_OUTPUTS = ByteSequenceOutputs.getSingleton(); + + static final BytesRef NO_OUTPUT = FST_OUTPUTS.getNoOutput(); /** Suggested default value for the {@code * minItemsInBlock} parameter to {@link @@ -597,14 +606,14 @@ public class BlockTreeTermsWriter extends FieldsConsumer { } // Write the top count entries on the pending stack as - // one or more blocks. Returns how many blocks were - // written. If the entry count is <= maxItemsPerBlock + // one or more blocks. If the entry count is <= maxItemsPerBlock // we just write a single block; else we break into // primary (initial) block and then one or more // following floor blocks: void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException { - if (prefixLength == 0 || count <= maxItemsInBlock) { + // System.out.println("writeBlocks count=" + count); + if (count <= maxItemsInBlock) { // Easy case: not floor block. Eg, prefix is "foo", // and we found 30 terms/sub-blocks starting w/ that // prefix, and minItemsInBlock <= 30 <= @@ -612,6 +621,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer { final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true); nonFloorBlock.compileIndex(null, scratchBytes); pending.add(nonFloorBlock); + // System.out.println(" 1 block"); } else { // Floor block case. Eg, prefix is "foo" but we // have 100 terms/sub-blocks starting w/ that @@ -768,6 +778,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer { floorBlocks.add(floorBlock); } curStart -= pendingCount; + // System.out.println(" floor=" + pendingCount); //System.out.println(" = " + pendingCount); pendingCount = 0; @@ -1041,12 +1052,12 @@ public class BlockTreeTermsWriter extends FieldsConsumer { // terms into "good" blocks; we don't save the // resulting FST: blockBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, - 0, 0, true, - true, Integer.MAX_VALUE, - noOutputs, - new FindBlocks(), false, - PackedInts.COMPACT, - true, 15); + 0, 0, true, + true, Integer.MAX_VALUE, + noOutputs, + new FindBlocks(), false, + PackedInts.COMPACT, + true, 15); this.longsSize = postingsWriter.setField(fieldInfo); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java new file mode 100644 index 00000000000..294b9572277 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java @@ -0,0 +1,176 @@ +package org.apache.lucene.codecs.blocktree; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FieldInfo; +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.BytesRef; +import org.apache.lucene.util.automaton.CompiledAutomaton; +import org.apache.lucene.util.fst.ByteSequenceOutputs; +import org.apache.lucene.util.fst.FST; + +/** BlockTree's implementation of {@link Terms}. */ +// public for CheckIndex: +public final class FieldReader extends Terms { + final long numTerms; + final FieldInfo fieldInfo; + final long sumTotalTermFreq; + final long sumDocFreq; + final int docCount; + final long indexStartFP; + final long rootBlockFP; + final BytesRef rootCode; + final BytesRef minTerm; + final BytesRef maxTerm; + final int longsSize; + final BlockTreeTermsReader parent; + + final FST index; + //private boolean DEBUG; + + FieldReader(BlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, + long indexStartFP, int longsSize, 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.indexStartFP = indexStartFP; + this.rootCode = rootCode; + this.longsSize = longsSize; + 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() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; + + if (indexIn != null) { + final IndexInput clone = indexIn.clone(); + //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name); + clone.seek(indexStartFP); + index = new FST<>(clone, ByteSequenceOutputs.getSingleton()); + + /* + 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(); + } + */ + } else { + index = null; + } + } + + @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*/ + // TODO: maybe push this into Terms? + public Stats computeStats() 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(TermsEnum reuse) 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 (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) { + throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead"); + } + return new IntersectTermsEnum(this, compiled, startTerm); + } + + /** Returns approximate RAM bytes used */ + public long ramBytesUsed() { + return ((index!=null)? index.sizeInBytes() : 0); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java new file mode 100644 index 00000000000..e910ed0db67 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java @@ -0,0 +1,484 @@ +package org.apache.lucene.codecs.blocktree; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.automaton.CompiledAutomaton; +import org.apache.lucene.util.automaton.RunAutomaton; +import org.apache.lucene.util.fst.ByteSequenceOutputs; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.Outputs; + +// NOTE: cannot seek! +final class IntersectTermsEnum extends TermsEnum { + final IndexInput in; + final static Outputs fstOutputs = ByteSequenceOutputs.getSingleton(); + + private IntersectTermsEnumFrame[] stack; + + @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc[] arcs = new FST.Arc[5]; + + final RunAutomaton runAutomaton; + final CompiledAutomaton compiledAutomaton; + + private IntersectTermsEnumFrame currentFrame; + + 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, CompiledAutomaton compiled, BytesRef startTerm) throws IOException { + // if (DEBUG) { + // System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef)); + // } + this.fr = fr; + runAutomaton = compiled.runAutomaton; + compiledAutomaton = compiled; + in = fr.parent.in.clone(); + stack = new IntersectTermsEnumFrame[5]; + for(int idx=0;idx(); + } + + if (fr.index == null) { + fstReader = null; + } else { + 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(runAutomaton.getInitialState()); + f.arc = arc; + f.outputPrefix = arc.output; + f.load(fr.rootCode); + + // for assert: + assert setSavedStartTerm(startTerm); + + currentFrame = f; + if (startTerm != null) { + seekToStartTerm(startTerm); + } + } + + // 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 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(); + } + arcs = next; + } + return arcs[ord]; + } + + private IntersectTermsEnumFrame pushFrame(int state) throws IOException { + final IntersectTermsEnumFrame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord); + + f.fp = f.fpOrig = currentFrame.lastSubFP; + f.prefix = currentFrame.prefix + currentFrame.suffix; + // if (DEBUG) System.out.println(" pushFrame state=" + state + " prefix=" + f.prefix); + 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 { + //if (DEBUG) System.out.println("BTIR.docFreq"); + currentFrame.decodeMetaData(); + //if (DEBUG) System.out.println(" return " + currentFrame.termState.docFreq); + return currentFrame.termState.docFreq; + } + + @Override + public long totalTermFreq() throws IOException { + currentFrame.decodeMetaData(); + return currentFrame.termState.totalTermFreq; + } + + @Override + public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException { + currentFrame.decodeMetaData(); + return fr.parent.postingsReader.docs(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { + if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { + // Positions were not indexed: + return null; + } + + currentFrame.decodeMetaData(); + return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags); + } + + private int getState() { + int state = currentFrame.state; + for(int idx=0;idx 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++]) { + // if (DEBUG) { + // System.out.println(" skip: common suffix mismatch (in prefix)"); + // } + continue nextTerm; + } + } + suffixBytesPos = currentFrame.startBytePos; + } else { + suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length; + } + + // Test overlapping suffix part: + final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length; + while (commonSuffixBytesPos < commonSuffixBytesPosEnd) { + if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) { + // if (DEBUG) { + // System.out.println(" skip: common suffix mismatch"); + // } + 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 prefix matches the automaton: + int state = currentFrame.state; + for (int idx=0;idx arc; + + final BlockTermState termState; + + // metadata buffer, holding monotonic values + public long[] longs; + // metadata buffer, holding general values + public byte[] bytes; + ByteArrayDataInput bytesReader; + + // Cumulative output so far + BytesRef outputPrefix; + + int startBytePos; + int suffix; + + private final IntersectTermsEnum ite; + + 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.longs = new long[ite.fr.longsSize]; + } + + void loadNextFloorBlock() throws IOException { + assert numFollowFloorBlocks > 0; + //if (DEBUG) System.out.println(" loadNextFoorBlock trans=" + transitions[transitionIndex]); + + do { + fp = fpOrig + (floorDataReader.readVLong() >>> 1); + numFollowFloorBlocks--; + // if (DEBUG) System.out.println(" skip floor block2! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks); + if (numFollowFloorBlocks != 0) { + nextFloorLabel = floorDataReader.readByte() & 0xff; + } else { + nextFloorLabel = 256; + } + // if (DEBUG) System.out.println(" nextFloorLabel=" + (char) nextFloorLabel); + } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin()); + + load(null); + } + + public void setState(int state) { + this.state = state; + transitionIndex = 0; + transitions = ite.compiledAutomaton.sortedTransitions[state]; + if (transitions.length != 0) { + curTransitionMax = transitions[0].getMax(); + } else { + curTransitionMax = -1; + } + } + + void load(BytesRef frameIndexData) throws IOException { + + // if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state)); + + if (frameIndexData != null && transitions.length != 0) { + // Floor frame + if (floorData.length < frameIndexData.length) { + this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)]; + } + System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length); + floorDataReader.reset(floorData, 0, frameIndexData.length); + // Skip first long -- has redundant fp, hasTerms + // flag, isFloor flag + final long code = floorDataReader.readVLong(); + if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) { + numFollowFloorBlocks = floorDataReader.readVInt(); + nextFloorLabel = floorDataReader.readByte() & 0xff; + // if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel); + + // If current state is accept, we must process + // first block in case it has empty suffix: + if (!ite.runAutomaton.isAccept(state)) { + // Maybe skip floor blocks: + while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) { + fp = fpOrig + (floorDataReader.readVLong() >>> 1); + numFollowFloorBlocks--; + // if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + 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: + code = ite.in.readVInt(); + isLeafBlock = (code & 1) != 0; + int numBytes = code >>> 1; + // if (DEBUG) System.out.println(" entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes); + if (suffixBytes.length < numBytes) { + suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + } + ite.in.readBytes(suffixBytes, 0, numBytes); + suffixesReader.reset(suffixBytes, 0, numBytes); + + // stats + 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); + metaDataUpto = 0; + + termState.termBlockOrd = 0; + nextEnt = 0; + + // metadata + numBytes = ite.in.readVInt(); + if (bytes == null) { + bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + bytesReader = new ByteArrayDataInput(); + } else 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 + + public boolean next() { + return isLeafBlock ? nextLeaf() : nextNonLeaf(); + } + + // Decodes next entry; returns true if it's a sub-block + public boolean 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 = suffixesReader.readVInt(); + startBytePos = suffixesReader.getPosition(); + suffixesReader.skipBytes(suffix); + return false; + } + + public boolean nextNonLeaf() { + //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++; + final int code = suffixesReader.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 - suffixesReader.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 + termState.docFreq = statsReader.readVInt(); + //if (DEBUG) System.out.println(" dF=" + state.docFreq); + if (ite.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) { + termState.totalTermFreq = termState.docFreq + statsReader.readVLong(); + //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); + } + // metadata + for (int i = 0; i < ite.fr.longsSize; i++) { + longs[i] = bytesReader.readVLong(); + } + ite.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ite.fr.fieldInfo, termState, absolute); + + metaDataUpto++; + absolute = false; + } + termState.termBlockOrd = metaDataUpto; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java new file mode 100644 index 00000000000..40fa1273ba9 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java @@ -0,0 +1,1052 @@ +package org.apache.lucene.codecs.blocktree; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.io.PrintStream; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.fst.ByteSequenceOutputs; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.Outputs; +import org.apache.lucene.util.fst.Util; + +/** Iterates through terms in this field */ +final class SegmentTermsEnum extends TermsEnum { + + // Lazy init: + IndexInput in; + + private SegmentTermsEnumFrame[] stack; + private final SegmentTermsEnumFrame staticFrame; + SegmentTermsEnumFrame currentFrame; + boolean termExists; + final FieldReader fr; + + private int targetBeforeCurrentLength; + + // static boolean DEBUG = false; + + 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 BytesRef term = new BytesRef(); + 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(); + } + + 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.in.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.indexNodeCount = fr.index.getNodeCount(); + stats.indexArcCount = fr.index.getArcCount(); + stats.indexNumBytes = fr.index.sizeInBytes(); + } + + 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) { + currentFrame.loadNextFloorBlock(); + stats.startBlock(currentFrame, true); + } 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.isFloor = false; + //currentFrame.hasTerms = true; + currentFrame.loadBlock(); + stats.startBlock(currentFrame, !currentFrame.isLastInFloor); + } else { + stats.term(term); + 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.length = 0; + + 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 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(); + } + 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 >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; + final SegmentTermsEnumFrame f = getFrame(1+currentFrame.ord); + f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0; + f.hasTermsOrig = f.hasTerms; + f.isFloor = (code & BlockTreeTermsWriter.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(); + } + } + + @Override + public boolean seekExact(final BytesRef target) throws IOException { + + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (term.bytes.length <= target.length) { + term.bytes = ArrayUtil.grow(term.bytes, 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.bytes[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 != BlockTreeTermsWriter.NO_OUTPUT) { + output = BlockTreeTermsWriter.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.bytes[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, BlockTreeTermsWriter.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.bytes[targetUpto] = (byte) targetLabel; + term.length = 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.bytes[targetUpto] = (byte) targetLabel; + // Aggregate output as we go: + assert arc.output != null; + if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) { + output = BlockTreeTermsWriter.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, BlockTreeTermsWriter.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.length = 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(final BytesRef target) throws IOException { + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (term.bytes.length <= target.length) { + term.bytes = ArrayUtil.grow(term.bytes, 1+target.length); + } + + assert clearEOF(); + + // if (DEBUG) { + // System.out.println("\nBTTR.seekCeil seg=" + fr.parent.segment + " target=" + fr.fieldInfo.name + ":" + target.utf8ToString() + " " + 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; + + // TOOD: 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.bytes[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); + // TOOD: 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 != BlockTreeTermsWriter.NO_OUTPUT) { + output = BlockTreeTermsWriter.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.bytes[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 + "); 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, BlockTreeTermsWriter.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) + " " + toHex(targetLabel)); + // } + + validIndexPrefix = currentFrame.prefix; + //validIndexPrefix = targetUpto; + + 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=" + brToString(term) + " " + 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) + " " + term); + //} + return result; + } + } else { + // Follow this arc + term.bytes[targetUpto] = (byte) targetLabel; + arc = nextArc; + // Aggregate output as we go: + assert arc.output != null; + if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) { + output = BlockTreeTermsWriter.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, BlockTreeTermsWriter.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.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.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< 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); + assert result; + } + + // Pop finished blocks + while (currentFrame.nextEnt == currentFrame.entCount) { + if (!currentFrame.isLastInFloor) { + currentFrame.loadNextFloorBlock(); + } else { + //if (DEBUG) System.out.println(" pop frame"); + if (currentFrame.ord == 0) { + //if (DEBUG) System.out.println(" return null"); + assert setEOF(); + term.length = 0; + 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); + 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.isFloor = false; + //currentFrame.hasTerms = true; + currentFrame.loadBlock(); + } else { + //if (DEBUG) System.out.println(" return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord); + return term; + } + } + } + + @Override + public BytesRef term() { + assert !eof; + return term; + } + + @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 DocsEnum docs(Bits skipDocs, DocsEnum 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.docs(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { + if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { + // Positions were not indexed: + return null; + } + + assert !eof; + currentFrame.decodeMetaData(); + return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.state, skipDocs, reuse, 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) != 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/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java new file mode 100644 index 00000000000..2f19c32b739 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java @@ -0,0 +1,745 @@ +package org.apache.lucene.codecs.blocktree; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.index.FieldInfo.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; + + // File pointer where this block was loaded from + long fp; + long fpOrig; + long fpEnd; + + byte[] suffixBytes = new byte[128]; + final ByteArrayDataInput suffixesReader = new ByteArrayDataInput(); + + byte[] statBytes = new byte[64]; + 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, holding monotonic values + public long[] longs; + // metadata buffer, holding general values + public byte[] bytes; + ByteArrayDataInput bytesReader; + + private final SegmentTermsEnum ste; + + 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.longs = new long[ste.fr.longsSize]; + } + + 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); + + // 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 + + // term suffixes: + code = ste.in.readVInt(); + isLeafBlock = (code & 1) != 0; + int numBytes = code >>> 1; + if (suffixBytes.length < numBytes) { + suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + } + ste.in.readBytes(suffixBytes, 0, numBytes); + suffixesReader.reset(suffixBytes, 0, numBytes); + + /*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 + 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); + 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 == null) { + bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + bytesReader = new ByteArrayDataInput(); + } else 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(); + 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); + } + */ + } + + public boolean next() { + return isLeafBlock ? nextLeaf() : nextNonLeaf(); + } + + // Decodes next entry; returns true if it's a sub-block + public boolean 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 = suffixesReader.readVInt(); + startBytePos = suffixesReader.getPosition(); + ste.term.length = prefix + suffix; + if (ste.term.bytes.length < ste.term.length) { + ste.term.grow(ste.term.length); + } + suffixesReader.readBytes(ste.term.bytes, prefix, suffix); + // A normal term + ste.termExists = true; + return false; + } + + public boolean nextNonLeaf() { + //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++; + final int code = suffixesReader.readVInt(); + suffix = code >>> 1; + startBytePos = suffixesReader.getPosition(); + ste.term.length = prefix + suffix; + if (ste.term.bytes.length < ste.term.length) { + 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 = suffixesReader.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: + + // stats + state.docFreq = statsReader.readVInt(); + //if (DEBUG) System.out.println(" dF=" + state.docFreq); + if (ste.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) { + state.totalTermFreq = state.docFreq + statsReader.readVLong(); + //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq); + } + // metadata + for (int i = 0; i < ste.fr.longsSize; i++) { + longs[i] = bytesReader.readVLong(); + } + ste.fr.parent.postingsReader.decodeTerm(longs, 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 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(" " + indexNodeCount + " nodes"); + out.println(" " + indexArcCount + " arcs"); + 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(" " + totalBlockSuffixBytes + " term suffix bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : "")); + 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 + + + + + + +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.PostingsBaseFormat} to implement the +postings. +

+ +

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

+ + diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java index f18ec89f877..93b21a77b01 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java @@ -19,14 +19,14 @@ package org.apache.lucene.codecs.lucene40; import java.io.IOException; -import org.apache.lucene.codecs.BlockTreeTermsReader; -import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; // javadocs +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.index.DocsEnum; // javadocs import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs import org.apache.lucene.index.FieldInfos; // javadocs diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java index 5633f0e3f2b..c0861ae3a78 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java @@ -20,8 +20,6 @@ package org.apache.lucene.codecs.lucene41; import java.io.IOException; -import org.apache.lucene.codecs.BlockTreeTermsReader; -import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; @@ -29,6 +27,8 @@ 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.DocsEnum; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.SegmentReadState; diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java index d6a3f80501d..1eb71648397 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java @@ -127,6 +127,8 @@ class BufferedUpdates { final AtomicInteger numTermDeletes = new AtomicInteger(); final AtomicInteger numNumericUpdates = new AtomicInteger(); final AtomicInteger numBinaryUpdates = new AtomicInteger(); + + // TODO: rename thes three: put "deleted" prefix in front: final Map terms = new HashMap<>(); final Map queries = new HashMap<>(); final List docIDs = new ArrayList<>(); diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index cba481157ac..c7f82fba794 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -28,9 +28,11 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import org.apache.lucene.codecs.BlockTreeTermsReader; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; +import org.apache.lucene.codecs.blocktree.FieldReader; +import org.apache.lucene.codecs.blocktree.Stats; import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.search.DocIdSetIterator; @@ -45,6 +47,7 @@ import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.LongBitSet; import org.apache.lucene.util.StringHelper; + /** * Basic tool and API to check the health of an index and * write a new segments file that removes reference to @@ -237,7 +240,7 @@ public class CheckIndex { * tree terms dictionary (this is only set if the * {@link PostingsFormat} for this segment uses block * tree. */ - public Map blockTreeStats = null; + public Map blockTreeStats = null; } /** @@ -1115,8 +1118,8 @@ public class CheckIndex { // docs got deleted and then merged away): } else { - if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) { - final BlockTreeTermsReader.Stats stats = ((BlockTreeTermsReader.FieldReader) fieldTerms).computeStats(); + if (fieldTerms instanceof FieldReader) { + final Stats stats = ((FieldReader) fieldTerms).computeStats(); assert stats != null; if (status.blockTreeStats == null) { status.blockTreeStats = new HashMap<>(); @@ -1249,7 +1252,7 @@ public class CheckIndex { } if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) { - for(Map.Entry ent : status.blockTreeStats.entrySet()) { + for(Map.Entry ent : status.blockTreeStats.entrySet()) { infoStream.println(" field \"" + ent.getKey() + "\":"); infoStream.println(" " + ent.getValue().toString().replace("\n", "\n ")); } @@ -1639,6 +1642,7 @@ public class CheckIndex { // Only agg stats if the doc is live: final boolean doStats = liveDocs == null || liveDocs.get(j); + if (doStats) { status.docCount++; } diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java index c7c7c6d5009..a060ee00ea8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java @@ -34,6 +34,7 @@ final class FreqProxTermsWriter extends TermsHash { } private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException { + // Process any pending Term deletes for this newly // flushed segment: if (state.segUpdates != null && state.segUpdates.terms.size() > 0) { @@ -57,11 +58,9 @@ final class FreqProxTermsWriter extends TermsHash { if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) { docsEnum = termsEnum.docs(null, docsEnum, 0); int delDocLimit = segDeletes.get(deleteTerm); + assert delDocLimit < DocsEnum.NO_MORE_DOCS; while (true) { int doc = docsEnum.nextDoc(); - if (doc == DocsEnum.NO_MORE_DOCS) { - break; - } if (doc < delDocLimit) { if (state.liveDocs == null) { state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount()); diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java b/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java index 0af6ba90d22..5ae23f8c0a4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java @@ -105,5 +105,6 @@ public class SegmentWriteState { this.segmentSuffix = segmentSuffix; segUpdates = state.segUpdates; delCountOnFlush = state.delCountOnFlush; + liveDocs = state.liveDocs; } } diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/PairOutputs.java b/lucene/core/src/java/org/apache/lucene/util/fst/PairOutputs.java index ca5fe763717..6d2699fd43b 100644 --- a/lucene/core/src/java/org/apache/lucene/util/fst/PairOutputs.java +++ b/lucene/core/src/java/org/apache/lucene/util/fst/PairOutputs.java @@ -61,6 +61,11 @@ public class PairOutputs extends Outputs> { public int hashCode() { return output1.hashCode() + output2.hashCode(); } + + @Override + public String toString() { + return "Pair(" + output1 + "," + output2 + ")"; + } }; public PairOutputs(Outputs outputs1, Outputs outputs2) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java index 32f17c04173..ecf4e9b05ca 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java @@ -967,73 +967,41 @@ public class TestTermsEnum extends LuceneTestCase { d.close(); } - /** Utility class to do efficient primary-key (only 1 doc contains the - * given term) lookups by segment, re-using the enums. This class is - * not thread safe, so it is the caller's job to create and use one - * instance of this per thread. Do not use this if a term may appear - * in more than one document! It will only return the first one it - * finds. */ - static class PerThreadPKLookup { - - private final TermsEnum[] termsEnums; - private final DocsEnum[] docsEnums; - private final Bits[] liveDocs; - private final int[] docBases; - private final int numSegs; - private final boolean hasDeletions; - - public PerThreadPKLookup(IndexReader r, String idFieldName) throws IOException { - - List leaves = new ArrayList<>(r.leaves()); - - // Larger segments are more likely to have the id, so we sort largest to smallest by numDocs: - Collections.sort(leaves, new Comparator() { - @Override - public int compare(AtomicReaderContext c1, AtomicReaderContext c2) { - return c2.reader().numDocs() - c1.reader().numDocs(); - } - }); - - termsEnums = new TermsEnum[leaves.size()]; - docsEnums = new DocsEnum[leaves.size()]; - liveDocs = new Bits[leaves.size()]; - docBases = new int[leaves.size()]; - int numSegs = 0; - boolean hasDeletions = false; - for(int i=0;i terms = new HashSet(); + int MAX_TERMS = atLeast(1000); + while (terms.size() < MAX_TERMS) { + terms.add(new BytesRef(TestUtil.randomSimpleString(random(), 1, 40))); } - - /** Returns docID if found, else -1. */ - public int lookup(BytesRef id) throws IOException { - for(int seg=0;seg termsList = new ArrayList<>(terms); + StringBuilder sb = new StringBuilder(); + for(int termCount=0;termCount N (see + * {@link IDVersionSegmentTermsEnum#seekExact(BytesRef,long)}. + * + *

This is most effective if the app assigns monotonically + * increasing global version to each indexed doc. Then, during + * indexing, use {@link + * IDVersionSegmentTermsEnum#seekExact(BytesRef,long)} (along with + * {@link LiveFieldValues}) to decide whether the document you are + * about to index was already indexed with a higher version, and skip + * it if so. + * + *

The field is effectively indexed as DOCS_ONLY and the docID is + * pulsed into the terms dictionary, but the user must feed in the + * version as a payload on the first token. + * + *

NOTE: term vectors cannot be indexed with this field (not that + * you should really ever want to do this). + * + * @lucene.experimental */ + +public class IDVersionPostingsFormat extends PostingsFormat { + + /** version must be >= this. */ + public static final long MIN_VERSION = 0; + + // TODO: we could delta encode instead, and keep the last bit: + + /** version must be <= this, because we encode with ZigZag. */ + public static final long MAX_VERSION = 0x3fffffffffffffffL; + + private final int minTermsInBlock; + private final int maxTermsInBlock; + + public IDVersionPostingsFormat() { + this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + public IDVersionPostingsFormat(int minTermsInBlock, int maxTermsInBlock) { + super("IDVersion"); + this.minTermsInBlock = minTermsInBlock; + this.maxTermsInBlock = maxTermsInBlock; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + PostingsWriterBase postingsWriter = new IDVersionPostingsWriter(state); + boolean success = false; + try { + FieldsConsumer ret = new VersionBlockTreeTermsWriter(state, + postingsWriter, + minTermsInBlock, + maxTermsInBlock); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsWriter); + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + PostingsReaderBase postingsReader = new IDVersionPostingsReader(); + boolean success = false; + try { + FieldsProducer ret = new VersionBlockTreeTermsReader(state.directory, + state.fieldInfos, + state.segmentInfo, + postingsReader, + state.context, + state.segmentSuffix); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsReader); + } + } + } + + public static long bytesToLong(BytesRef bytes) { + return ((bytes.bytes[bytes.offset]&0xFFL) << 56) | + ((bytes.bytes[bytes.offset+1]&0xFFL) << 48) | + ((bytes.bytes[bytes.offset+2]&0xFFL) << 40) | + ((bytes.bytes[bytes.offset+3]&0xFFL) << 32) | + ((bytes.bytes[bytes.offset+4]&0xFFL) << 24) | + ((bytes.bytes[bytes.offset+5]&0xFFL) << 16) | + ((bytes.bytes[bytes.offset+6]&0xFFL) << 8) | + (bytes.bytes[bytes.offset+7]&0xFFL); + } + + public static void longToBytes(long v, BytesRef bytes) { + if (v > MAX_VERSION || v < MIN_VERSION) { + throw new IllegalArgumentException("version must be >= MIN_VERSION=" + MIN_VERSION + " and <= MAX_VERSION=" + MAX_VERSION + " (got: " + v + ")"); + } + bytes.offset = 0; + bytes.length = 8; + bytes.bytes[0] = (byte) (v >> 56); + bytes.bytes[1] = (byte) (v >> 48); + bytes.bytes[2] = (byte) (v >> 40); + bytes.bytes[3] = (byte) (v >> 32); + bytes.bytes[4] = (byte) (v >> 24); + bytes.bytes[5] = (byte) (v >> 16); + bytes.bytes[6] = (byte) (v >> 8); + bytes.bytes[7] = (byte) v; + assert bytesToLong(bytes) == v: bytesToLong(bytes) + " vs " + v + " bytes=" + bytes; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java new file mode 100644 index 00000000000..54887c5610a --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java @@ -0,0 +1,102 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.Bits; + +final class IDVersionPostingsReader extends PostingsReaderBase { + + @Override + public void init(IndexInput termsIn) throws IOException { + // Make sure we are talking to the matching postings writer + CodecUtil.checkHeader(termsIn, + IDVersionPostingsWriter.TERMS_CODEC, + IDVersionPostingsWriter.VERSION_START, + IDVersionPostingsWriter.VERSION_CURRENT); + } + + @Override + public BlockTermState newTermState() { + return new IDVersionTermState(); + } + + @Override + public void close() throws IOException { + } + + @Override + public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute) + throws IOException { + final IDVersionTermState termState = (IDVersionTermState) _termState; + termState.docID = in.readVInt(); + if (absolute) { + termState.idVersion = in.readVLong(); + } else { + termState.idVersion += BitUtil.zigZagDecode(in.readVLong()); + } + } + + @Override + public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException { + SingleDocsEnum docsEnum; + + if (reuse instanceof SingleDocsEnum) { + docsEnum = (SingleDocsEnum) reuse; + } else { + docsEnum = new SingleDocsEnum(); + } + docsEnum.reset(((IDVersionTermState) termState).docID, liveDocs); + + return docsEnum; + } + + @Override + public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, + DocsAndPositionsEnum reuse, int flags) { + SingleDocsAndPositionsEnum posEnum; + + if (reuse instanceof SingleDocsAndPositionsEnum) { + posEnum = (SingleDocsAndPositionsEnum) reuse; + } else { + posEnum = new SingleDocsAndPositionsEnum(); + } + IDVersionTermState termState = (IDVersionTermState) _termState; + posEnum.reset(termState.docID, termState.idVersion, liveDocs); + return posEnum; + } + + @Override + public long ramBytesUsed() { + return 0; + } + + @Override + public void checkIntegrity() throws IOException { + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java new file mode 100644 index 00000000000..9dc44143cb6 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java @@ -0,0 +1,169 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PushPostingsWriterBase; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; + +final class IDVersionPostingsWriter extends PushPostingsWriterBase { + + final static String TERMS_CODEC = "IDVersionPostingsWriterTerms"; + + // Increment version to change it + final static int VERSION_START = 0; + final static int VERSION_CURRENT = VERSION_START; + + final static IDVersionTermState emptyState = new IDVersionTermState(); + IDVersionTermState lastState; + + int lastDocID; + private int lastPosition; + private long lastVersion; + + private final SegmentWriteState state; + + public IDVersionPostingsWriter(SegmentWriteState state) { + this.state = state; + } + + @Override + public BlockTermState newTermState() { + return new IDVersionTermState(); + } + + @Override + public void init(IndexOutput termsOut) throws IOException { + CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT); + } + + @Override + public int setField(FieldInfo fieldInfo) { + super.setField(fieldInfo); + if (fieldInfo.getIndexOptions() != FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + throw new IllegalArgumentException("field must be index using IndexOptions.DOCS_AND_FREQS_AND_POSITIONS"); + } + // LUCENE-5693: because CheckIndex cross-checks term vectors with postings even for deleted docs, and because our PF only indexes the + // non-deleted documents on flush, CheckIndex will see this as corruption: + if (fieldInfo.hasVectors()) { + throw new IllegalArgumentException("field cannot index term vectors: CheckIndex will report this as index corruption"); + } + lastState = emptyState; + return 0; + } + + @Override + public void startTerm() { + lastDocID = -1; + } + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + // TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush: + if (state.liveDocs != null && state.liveDocs.get(docID) == false) { + return; + } + if (lastDocID != -1) { + throw new IllegalArgumentException("term appears in more than one document"); + } + if (termDocFreq != 1) { + throw new IllegalArgumentException("term appears more than once in the document"); + } + + lastDocID = docID; + lastPosition = -1; + lastVersion = -1; + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException { + if (lastDocID == -1) { + // Doc is deleted; skip it + return; + } + if (lastPosition != -1) { + throw new IllegalArgumentException("term appears more than once in document"); + } + lastPosition = position; + if (payload == null) { + throw new IllegalArgumentException("token doens't have a payload"); + } + if (payload.length != 8) { + throw new IllegalArgumentException("payload.length != 8 (got " + payload.length + ")"); + } + + lastVersion = IDVersionPostingsFormat.bytesToLong(payload); + if (lastVersion < IDVersionPostingsFormat.MIN_VERSION) { + throw new IllegalArgumentException("version must be >= MIN_VERSION=" + IDVersionPostingsFormat.MIN_VERSION + " (got: " + lastVersion + "; payload=" + payload + ")"); + } + if (lastVersion > IDVersionPostingsFormat.MAX_VERSION) { + throw new IllegalArgumentException("version must be <= MAX_VERSION=" + IDVersionPostingsFormat.MAX_VERSION + " (got: " + lastVersion + "; payload=" + payload + ")"); + } + } + + @Override + public void finishDoc() throws IOException { + if (lastDocID == -1) { + // Doc is deleted; skip it + return; + } + if (lastPosition == -1) { + throw new IllegalArgumentException("missing addPosition"); + } + } + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(BlockTermState _state) throws IOException { + if (lastDocID == -1) { + return; + } + IDVersionTermState state = (IDVersionTermState) _state; + assert state.docFreq > 0; + + state.docID = lastDocID; + state.idVersion = lastVersion; + } + + private long lastEncodedVersion; + + @Override + public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException { + IDVersionTermState state = (IDVersionTermState) _state; + out.writeVInt(state.docID); + if (absolute) { + out.writeVLong(state.idVersion); + } else { + long delta = state.idVersion - lastEncodedVersion; + out.writeVLong(BitUtil.zigZagEncode(delta)); + } + lastEncodedVersion = state.idVersion; + } + + @Override + public void close() throws IOException { + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java new file mode 100644 index 00000000000..bca8027b142 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java @@ -0,0 +1,1071 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.io.PrintStream; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.PairOutputs.Pair; +import org.apache.lucene.util.fst.Util; + +/** Iterates through terms in this field; this class is public so users + * can cast it to call {@link #seekExact(BytesRef, long)} for + * optimistic-concurreny, and also {@link #getVersion} to get the + * version of the currently seek'd term. */ +public final class IDVersionSegmentTermsEnum extends TermsEnum { + + // Lazy init: + IndexInput in; + + // static boolean DEBUG = false; + + private IDVersionSegmentTermsEnumFrame[] stack; + private final IDVersionSegmentTermsEnumFrame staticFrame; + IDVersionSegmentTermsEnumFrame currentFrame; + boolean termExists; + final VersionFieldReader fr; + + private int targetBeforeCurrentLength; + + private final ByteArrayDataInput scratchReader = new ByteArrayDataInput(); + + // What prefix of the current term was present in the index: + private int validIndexPrefix; + + // assert only: + private boolean eof; + + final BytesRef term = new BytesRef(); + private final FST.BytesReader fstReader; + + @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc>[] arcs = + new FST.Arc[1]; + + IDVersionSegmentTermsEnum(VersionFieldReader fr) throws IOException { + this.fr = fr; + + //if (DEBUG) System.out.println("BTTR.init seg=" + segment); + stack = new IDVersionSegmentTermsEnumFrame[0]; + + // Used to hold seek by TermState, or cached seek + staticFrame = new IDVersionSegmentTermsEnumFrame(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(); + } + + 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 = staticFrame; + //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.in.clone(); + } + } + + private IDVersionSegmentTermsEnumFrame getFrame(int ord) throws IOException { + if (ord >= stack.length) { + final IDVersionSegmentTermsEnumFrame[] next = new IDVersionSegmentTermsEnumFrame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; + System.arraycopy(stack, 0, next, 0, stack.length); + for(int stackOrd=stack.length;stackOrd> 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(); + } + arcs = next; + } + return arcs[ord]; + } + + // Pushes a frame we seek'd to + IDVersionSegmentTermsEnumFrame pushFrame(FST.Arc> arc, Pair frameData, int length) throws IOException { + scratchReader.reset(frameData.output1.bytes, frameData.output1.offset, frameData.output1.length); + final long code = scratchReader.readVLong(); + final long fpSeek = code >>> VersionBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; + final IDVersionSegmentTermsEnumFrame f = getFrame(1+currentFrame.ord); + f.maxIDVersion = Long.MAX_VALUE - frameData.output2; + f.hasTerms = (code & VersionBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0; + f.hasTermsOrig = f.hasTerms; + f.isFloor = (code & VersionBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0; + if (f.isFloor) { + f.setFloorData(scratchReader, frameData.output1); + } + pushFrame(arc, fpSeek, length); + + return f; + } + + // Pushes next'd frame or seek'd frame; we later + // lazy-load the frame only when needed + IDVersionSegmentTermsEnumFrame pushFrame(FST.Arc> arc, long fp, int length) throws IOException { + final IDVersionSegmentTermsEnumFrame 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) { + 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; + } + + @Override + public boolean seekExact(final BytesRef target) throws IOException { + return seekExact(target, 0); + } + + // 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(); + } + } + + /** Get the version of the currently seek'd term; only valid if we are + * positioned. */ + public long getVersion() { + return ((IDVersionTermState) currentFrame.state).idVersion; + } + + /** Optimized version of {@link #seekExact(BytesRef)} that can + * sometimes fail-fast if the version indexed with the requested ID + * is less than the specified minIDVersion. Applications that index + * a monotonically increasing global version with each document can + * use this for fast optimistic concurrency. */ + public boolean seekExact(final BytesRef target, long minIDVersion) throws IOException { + + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (term.bytes.length <= target.length) { + term.bytes = ArrayUtil.grow(term.bytes, 1+target.length); + } + + assert clearEOF(); + + // if (DEBUG) { + // System.out.println("\nBTTR.seekExact seg=" + fr.parent.segment + " target=" + fr.fieldInfo.name + ":" + brToString(target) + " minIDVersion=" + minIDVersion + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=" + validIndexPrefix); + // printSeekState(System.out); + // } + + FST.Arc> arc; + int targetUpto; + Pair output; + + long startFrameFP = currentFrame.fp; + + targetBeforeCurrentLength = currentFrame.ord; + + boolean changed = false; + + // TODO: we could stop earlier w/ the version check, every time we traverse an index arc we can check? + + 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; + + IDVersionSegmentTermsEnumFrame lastFrame = stack[0]; + assert validIndexPrefix <= term.length: "validIndexPrefix=" + validIndexPrefix + " term.length=" + term.length + " seg=" + fr.parent.segment; + + 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.bytes[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]; + //if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) { + //System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF)); + //} + 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 != VersionBlockTreeTermsWriter.NO_OUTPUT) { + output = VersionBlockTreeTermsWriter.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.bytes[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 + "; targetUpto=" + targetUpto); + // } + 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; + changed = true; + // 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 (currentFrame.maxIDVersion < minIDVersion) { + // The max version for all terms in this block is lower than the minVersion + // if (DEBUG) { + // System.out.println(" target is same as current maxIDVersion=" + currentFrame.maxIDVersion + " is < minIDVersion=" + minIDVersion + "; return false"); + // } + return false; + } + + currentFrame.decodeMetaData(); + if (((IDVersionTermState) currentFrame.state).idVersion < minIDVersion) { + // This term's version is lower than the minVersion + // if (DEBUG) { + // System.out.println(" target is same as current but version=" + ((IDVersionTermState) currentFrame.state).idVersion + " is < minIDVersion=" + minIDVersion + "; return false"); + // } + return false; + } + // System.out.println(" term version=" + ((IDVersionTermState) currentFrame.state).idVersion + " frame version=" + currentFrame.maxIDVersion + " frame ord=" + currentFrame.ord); + + // 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]); + //System.out.println("first arc=" + arc); + + // 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, VersionBlockTreeTermsWriter.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 + " termExists=" + termExists); + // } + + // 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) + " " + Integer.toHexString(targetLabel) + " termExists=" + termExists); + // } + + validIndexPrefix = currentFrame.prefix; + //validIndexPrefix = targetUpto; + + currentFrame.scanToFloorFrame(target); + + if (!currentFrame.hasTerms) { + termExists = false; + term.bytes[targetUpto] = (byte) targetLabel; + term.length = 1+targetUpto; + // if (DEBUG) { + // System.out.println(" FAST NOT_FOUND term=" + brToString(term)); + // } + return false; + } + //System.out.println(" check maxVersion=" + currentFrame.maxIDVersion + " vs " + minIDVersion); + + // if (DEBUG) { + // System.out.println(" frame.maxIDVersion=" + currentFrame.maxIDVersion + " vs minIDVersion=" + minIDVersion); + // } + + if (currentFrame.maxIDVersion < minIDVersion) { + // The max version for all terms in this block is lower than the minVersion + if (currentFrame.fp != startFrameFP || changed) { + //if (targetUpto+1 > term.length) { + termExists = false; + term.bytes[targetUpto] = (byte) targetLabel; + term.length = 1+targetUpto; + // if (DEBUG) { + // System.out.println(" reset current term"); + // } + validIndexPrefix = Math.min(validIndexPrefix, term.length); + } + //if (currentFrame.ord != startFrameOrd) { + //termExists = false; + //} + // if (DEBUG) { + // System.out.println(" FAST version NOT_FOUND term=" + brToString(term) + " targetUpto=" + targetUpto + " currentFrame.maxIDVersion=" + currentFrame.maxIDVersion + " validIndexPrefix=" + validIndexPrefix + " startFrameFP=" + startFrameFP + " vs " + currentFrame.fp + " termExists=" + termExists); + // } + return false; + } + + currentFrame.loadBlock(); + + // if (DEBUG) { + // System.out.println(" scan currentFrame ord=" + currentFrame.ord); + // } + final SeekStatus result = currentFrame.scanToTerm(target, true); + if (result == SeekStatus.FOUND) { + currentFrame.decodeMetaData(); + if (((IDVersionTermState) currentFrame.state).idVersion < minIDVersion) { + // This term's version is lower than the minVersion + // if (DEBUG) { + // System.out.println(" return NOT_FOUND: idVersion=" + ((IDVersionTermState) currentFrame.state).idVersion + " vs minIDVersion=" + minIDVersion); + // } + return false; + } + + // 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; + if (term.bytes[targetUpto] != (byte) targetLabel) { + // if (DEBUG) { + // System.out.println(" now set termExists=false targetUpto=" + targetUpto + " term=" + term.bytes[targetUpto] + " targetLabel=" + targetLabel); + // } + changed = true; + term.bytes[targetUpto] = (byte) targetLabel; + termExists = false; + } + // Aggregate output as we go: + assert arc.output != null; + if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) { + output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output); + } + + // if (DEBUG) { + // System.out.println(" index: follow label=" + (char) ((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, VersionBlockTreeTermsWriter.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.length = targetUpto; + // if (DEBUG) { + // System.out.println(" FAST NOT_FOUND term=" + brToString(term)); + // } + return false; + } + + // if (DEBUG) { + // System.out.println(" frame.maxIDVersion=" + currentFrame.maxIDVersion + " vs minIDVersion=" + minIDVersion); + // } + + if (currentFrame.maxIDVersion < minIDVersion) { + // The max version for all terms in this block is lower than the minVersion + termExists = false; + term.length = targetUpto; + 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); + // } + currentFrame.decodeMetaData(); + if (((IDVersionTermState) currentFrame.state).idVersion < minIDVersion) { + // This term's version is lower than the minVersion + return false; + } + return true; + } else { + // if (DEBUG) { + // System.out.println(" got result " + result + "; return NOT_FOUND term=" + term.utf8ToString()); + // } + + return false; + } + } + + @Override + public SeekStatus seekCeil(final BytesRef target) throws IOException { + if (fr.index == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (term.bytes.length <= target.length) { + term.bytes = ArrayUtil.grow(term.bytes, 1+target.length); + } + + assert clearEOF(); + + //if (DEBUG) { + //System.out.println("\nBTTR.seekCeil seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix= " + validIndexPrefix); + //printSeekState(); + //} + + FST.Arc> arc; + int targetUpto; + Pair 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; + + IDVersionSegmentTermsEnumFrame lastFrame = stack[0]; + assert validIndexPrefix <= term.length; + + final int targetLimit = Math.min(target.length, validIndexPrefix); + + int cmp = 0; + + // TOOD: 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.bytes[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); + // TOOD: 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 != VersionBlockTreeTermsWriter.NO_OUTPUT) { + output = VersionBlockTreeTermsWriter.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.bytes[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 + "); 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, VersionBlockTreeTermsWriter.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) + " " + toHex(targetLabel)); + // } + + validIndexPrefix = currentFrame.prefix; + //validIndexPrefix = targetUpto; + + 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=" + brToString(term) + " " + 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) + " " + term); + //} + return result; + } + } else { + // Follow this arc + term.bytes[targetUpto] = (byte) targetLabel; + arc = nextArc; + // Aggregate output as we go: + assert arc.output != null; + if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) { + output = VersionBlockTreeTermsWriter.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, VersionBlockTreeTermsWriter.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.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) { + IDVersionSegmentTermsEnumFrame f = getFrame(ord); + assert f != null; + final BytesRef prefix = new BytesRef(term.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=" + brToString(prefix) + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp< 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.output1.bytes, output.output1.offset, output.output1.length); + final long codeOrig = reader.readVLong(); + final long code = (f.fp << VersionBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? VersionBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? VersionBlockTreeTermsWriter.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=" + segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix); + //printSeekState(); + //} + + 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); + assert result; + } + + // Pop finished blocks + while (currentFrame.nextEnt == currentFrame.entCount) { + if (!currentFrame.isLastInFloor) { + currentFrame.loadNextFloorBlock(); + } else { + //if (DEBUG) System.out.println(" pop frame"); + if (currentFrame.ord == 0) { + //if (DEBUG) System.out.println(" return null"); + assert setEOF(); + term.length = 0; + 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); + 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.isFloor = false; + //currentFrame.hasTerms = true; + currentFrame.loadBlock(); + } else { + //if (DEBUG) System.out.println(" return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord); + return term; + } + } + } + + @Override + public BytesRef term() { + assert !eof; + return term; + } + + @Override + public int docFreq() throws IOException { + assert !eof; + return 1; + } + + @Override + public long totalTermFreq() throws IOException { + assert !eof; + return 1; + } + + @Override + public DocsEnum docs(Bits skipDocs, DocsEnum 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.docs(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException { + if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) { + // Positions were not indexed: + return null; + } + + assert !eof; + currentFrame.decodeMetaData(); + return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.state, skipDocs, reuse, 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) != 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(); + } + + @Override + public String toString() { + return "IDVersionSegmentTermsEnum(seg=" + fr.parent.segment + ")"; + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java new file mode 100644 index 00000000000..fa64056c7f8 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnumFrame.java @@ -0,0 +1,738 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.index.FieldInfo.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; +import org.apache.lucene.util.fst.PairOutputs.Pair; + +final class IDVersionSegmentTermsEnumFrame { + // Our index in stack[]: + final int ord; + + boolean hasTerms; + boolean hasTermsOrig; + boolean isFloor; + + // static boolean DEBUG = IDVersionSegmentTermsEnum.DEBUG; + + /** Highest version of any term in this block. */ + long maxIDVersion; + + FST.Arc> arc; + + // File pointer where this block was loaded from + long fp; + long fpOrig; + long fpEnd; + + byte[] suffixBytes = new byte[128]; + final ByteArrayDataInput suffixesReader = 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, holding monotonic values + public long[] longs; + // metadata buffer, holding general values + public byte[] bytes; + ByteArrayDataInput bytesReader; + + private final IDVersionSegmentTermsEnum ste; + + public IDVersionSegmentTermsEnumFrame(IDVersionSegmentTermsEnum ste, int ord) throws IOException { + this.ste = ste; + this.ord = ord; + this.state = ste.fr.parent.postingsReader.newTermState(); + this.state.totalTermFreq = -1; + this.longs = new long[ste.fr.longsSize]; + } + + 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); + + // 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 + + // term suffixes: + code = ste.in.readVInt(); + isLeafBlock = (code & 1) != 0; + int numBytes = code >>> 1; + if (suffixBytes.length < numBytes) { + suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + } + ste.in.readBytes(suffixBytes, 0, numBytes); + suffixesReader.reset(suffixBytes, 0, numBytes); + + /*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); + } + }*/ + + 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 == null) { + bytes = new byte[ArrayUtil.oversize(numBytes, 1)]; + bytesReader = new ByteArrayDataInput(); + } else 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(); + 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); + } + */ + } + + public boolean next() { + return isLeafBlock ? nextLeaf() : nextNonLeaf(); + } + + // Decodes next entry; returns true if it's a sub-block + public boolean 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 = suffixesReader.readVInt(); + startBytePos = suffixesReader.getPosition(); + ste.term.length = prefix + suffix; + if (ste.term.bytes.length < ste.term.length) { + ste.term.grow(ste.term.length); + } + suffixesReader.readBytes(ste.term.bytes, prefix, suffix); + // A normal term + ste.termExists = true; + return false; + } + + public boolean nextNonLeaf() { + // 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++; + final int code = suffixesReader.readVInt(); + suffix = code >>> 1; + startBytePos = suffixesReader.getPosition(); + ste.term.length = prefix + suffix; + if (ste.term.bytes.length < ste.term.length) { + 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 = suffixesReader.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=" + ((char) targetLabel) + " vs nextFloorLabel=" + ((char) 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=" + ((char) nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks); + // } + + isLastInFloor = numFollowFloorBlocks == 1; + numFollowFloorBlocks--; + + if (isLastInFloor) { + nextFloorLabel = 256; + // if (DEBUG) { + // System.out.println(" stop! last block nextFloorLabel=" + ((char) nextFloorLabel)); + // } + break; + } else { + nextFloorLabel = floorDataReader.readByte() & 0xff; + if (targetLabel < nextFloorLabel) { + // if (DEBUG) { + // System.out.println(" stop! nextFloorLabel=" + ((char) 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=" + ste.fr.parent.segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd); + + assert nextEnt >= 0; + + // lazily catch up on metadata decode: + final int limit = getTermBlockOrd(); + boolean absolute = metaDataUpto == 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 + state.docFreq = 1; + state.totalTermFreq = 1; + //if (DEBUG) System.out.println(" dF=" + state.docFreq); + // metadata + for (int i = 0; i < ste.fr.longsSize; i++) { + longs[i] = bytesReader.readVLong(); + } + ste.fr.parent.postingsReader.decodeTerm(longs, 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 fields = new TreeMap<>(); + + /** File offset where the directory starts in the terms file. */ + private long dirOffset; + + /** File offset where the directory starts in the index file. */ + private long indexDirOffset; + + final String segment; + + private final int version; + + /** Sole constructor. */ + public VersionBlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info, + PostingsReaderBase postingsReader, IOContext ioContext, + String segmentSuffix) + throws IOException { + + this.postingsReader = postingsReader; + + this.segment = info.name; + in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VersionBlockTreeTermsWriter.TERMS_EXTENSION), + ioContext); + + boolean success = false; + IndexInput indexIn = null; + + try { + version = readHeader(in); + indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VersionBlockTreeTermsWriter.TERMS_INDEX_EXTENSION), + ioContext); + int indexVersion = readIndexHeader(indexIn); + if (indexVersion != version) { + throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion); + } + + // verify + CodecUtil.checksumEntireFile(indexIn); + + // Have PostingsReader init itself + postingsReader.init(in); + + // Read per-field details + seekDir(in, dirOffset); + seekDir(indexIn, indexDirOffset); + + final int numFields = in.readVInt(); + if (numFields < 0) { + throw new CorruptIndexException("invalid numFields: " + numFields + " (resource=" + in + ")"); + } + + for(int i=0;i= 0; + final int numBytes = in.readVInt(); + final BytesRef code = new BytesRef(new byte[numBytes]); + in.readBytes(code.bytes, 0, numBytes); + code.length = numBytes; + final long version = in.readVLong(); + final Pair rootCode = VersionBlockTreeTermsWriter.FST_OUTPUTS.newPair(code, version); + final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + assert fieldInfo != null: "field=" + field; + final long sumTotalTermFreq = numTerms; + final long sumDocFreq = numTerms; + assert numTerms <= Integer.MAX_VALUE; + final int docCount = (int) numTerms; + final int longsSize = in.readVInt(); + + BytesRef minTerm = readBytesRef(in); + BytesRef maxTerm = readBytesRef(in); + if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs + throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")"); + } + if (sumDocFreq < docCount) { // #postings must be >= #docs with field + throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")"); + } + if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings + throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")"); + } + final long indexStartFP = indexIn.readVLong(); + VersionFieldReader previous = fields.put(fieldInfo.name, + new VersionFieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, + indexStartFP, longsSize, indexIn, minTerm, maxTerm)); + if (previous != null) { + throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")"); + } + } + indexIn.close(); + + success = true; + } finally { + if (!success) { + // this.close() will close in: + IOUtils.closeWhileHandlingException(indexIn, this); + } + } + } + + private static BytesRef readBytesRef(IndexInput in) throws IOException { + BytesRef bytes = new BytesRef(); + bytes.length = in.readVInt(); + bytes.bytes = new byte[bytes.length]; + in.readBytes(bytes.bytes, 0, bytes.length); + return bytes; + } + + /** Reads terms file header. */ + private int readHeader(IndexInput input) throws IOException { + int version = CodecUtil.checkHeader(input, VersionBlockTreeTermsWriter.TERMS_CODEC_NAME, + VersionBlockTreeTermsWriter.VERSION_START, + VersionBlockTreeTermsWriter.VERSION_CURRENT); + return version; + } + + /** Reads index file header. */ + private int readIndexHeader(IndexInput input) throws IOException { + int version = CodecUtil.checkHeader(input, VersionBlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME, + VersionBlockTreeTermsWriter.VERSION_START, + VersionBlockTreeTermsWriter.VERSION_CURRENT); + return version; + } + + /** Seek {@code input} to the directory offset. */ + private void seekDir(IndexInput input, long dirOffset) + throws IOException { + input.seek(input.length() - CodecUtil.footerLength() - 8); + dirOffset = input.readLong(); + input.seek(dirOffset); + } + + // for debugging + // private static String toHex(int v) { + // return "0x" + Integer.toHexString(v); + // } + + @Override + public void close() throws IOException { + try { + IOUtils.close(in, postingsReader); + } finally { + // Clear so refs to terms index is GCable even if + // app hangs onto us: + fields.clear(); + } + } + + @Override + public Iterator iterator() { + return Collections.unmodifiableSet(fields.keySet()).iterator(); + } + + @Override + public Terms terms(String field) throws IOException { + assert field != null; + return fields.get(field); + } + + @Override + public int size() { + return fields.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 sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0); + for(VersionFieldReader reader : fields.values()) { + sizeInByes += reader.ramBytesUsed(); + } + return sizeInByes; + } + + @Override + public void checkIntegrity() throws IOException { + // term dictionary + CodecUtil.checksumEntireFile(in); + + // postings + postingsReader.checkIntegrity(); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java new file mode 100644 index 00000000000..24b656f77e0 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java @@ -0,0 +1,1032 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; +import org.apache.lucene.index.FieldInfo.IndexOptions; +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.SegmentWriteState; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.IntsRef; +import org.apache.lucene.util.fst.Builder; +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.NoOutputs; +import org.apache.lucene.util.fst.PairOutputs.Pair; +import org.apache.lucene.util.fst.PairOutputs; +import org.apache.lucene.util.fst.PositiveIntOutputs; +import org.apache.lucene.util.fst.Util; +import org.apache.lucene.util.packed.PackedInts; + +/* + 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. +*/ + +/** + * 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 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. + * + * @lucene.experimental + */ + +final class VersionBlockTreeTermsWriter extends FieldsConsumer { + + // private static boolean DEBUG = IDVersionSegmentTermsEnum.DEBUG; + + static final PairOutputs FST_OUTPUTS = new PairOutputs<>(ByteSequenceOutputs.getSingleton(), + PositiveIntOutputs.getSingleton()); + + static final Pair NO_OUTPUT = FST_OUTPUTS.getNoOutput(); + + /** Suggested default value for the {@code + * minItemsInBlock} parameter to {@link + * #VersionBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */ + public final static int DEFAULT_MIN_BLOCK_SIZE = 25; + + /** Suggested default value for the {@code + * maxItemsInBlock} parameter to {@link + * #VersionBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */ + public final static int DEFAULT_MAX_BLOCK_SIZE = 48; + + //public final static boolean DEBUG = false; + //private final static boolean SAVE_DOT_FILES = false; + + 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 = "tiv"; + final static String TERMS_CODEC_NAME = "VERSION_BLOCK_TREE_TERMS_DICT"; + + /** 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 = "tipv"; + final static String TERMS_INDEX_CODEC_NAME = "VERSION_BLOCK_TREE_TERMS_INDEX"; + + private final IndexOutput out; + private final IndexOutput indexOut; + final int maxDoc; + final int minItemsInBlock; + final int maxItemsInBlock; + + final PostingsWriterBase postingsWriter; + final FieldInfos fieldInfos; + + private static class FieldMetaData { + public final FieldInfo fieldInfo; + public final Pair rootCode; + public final long numTerms; + public final long indexStartFP; + private final int longsSize; + public final BytesRef minTerm; + public final BytesRef maxTerm; + + public FieldMetaData(FieldInfo fieldInfo, Pair rootCode, long numTerms, long indexStartFP, int longsSize, + BytesRef minTerm, BytesRef maxTerm) { + assert numTerms > 0; + this.fieldInfo = fieldInfo; + assert rootCode != null: "field=" + fieldInfo.name + " numTerms=" + numTerms; + this.rootCode = rootCode; + this.indexStartFP = indexStartFP; + this.numTerms = numTerms; + this.longsSize = longsSize; + this.minTerm = minTerm; + this.maxTerm = maxTerm; + } + } + + private final List fields = new ArrayList<>(); + // private final String segment; + + /** 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 VersionBlockTreeTermsWriter( + SegmentWriteState state, + PostingsWriterBase postingsWriter, + int minItemsInBlock, + int maxItemsInBlock) + throws IOException + { + if (minItemsInBlock <= 1) { + throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock); + } + if (maxItemsInBlock <= 0) { + throw new IllegalArgumentException("maxItemsInBlock must be >= 1; got " + maxItemsInBlock); + } + 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); + } + + maxDoc = state.segmentInfo.getDocCount(); + + final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION); + out = state.directory.createOutput(termsFileName, state.context); + boolean success = false; + IndexOutput indexOut = null; + try { + fieldInfos = state.fieldInfos; + this.minItemsInBlock = minItemsInBlock; + this.maxItemsInBlock = maxItemsInBlock; + CodecUtil.writeHeader(out, TERMS_CODEC_NAME, VERSION_CURRENT); + + //DEBUG = state.segmentName.equals("_4a"); + + final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION); + indexOut = state.directory.createOutput(termsIndexFileName, state.context); + CodecUtil.writeHeader(indexOut, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT); + + this.postingsWriter = postingsWriter; + // segment = state.segmentInfo.name; + + // System.out.println("BTW.init seg=" + state.segmentName); + + postingsWriter.init(out); // have consumer write its format/header + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(out, indexOut); + } + } + this.indexOut = indexOut; + } + + /** Writes the terms file trailer. */ + private void writeTrailer(IndexOutput out, long dirStart) throws IOException { + out.writeLong(dirStart); + } + + /** Writes the index file trailer. */ + private void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException { + indexOut.writeLong(dirStart); + } + + @Override + public void write(Fields fields) throws IOException { + + String lastField = null; + for(String field : fields) { + assert lastField == null || lastField.compareTo(field) < 0; + lastField = field; + + Terms terms = fields.terms(field); + if (terms == null) { + continue; + } + + TermsEnum termsEnum = terms.iterator(null); + + TermsWriter termsWriter = new TermsWriter(fieldInfos.fieldInfo(field)); + BytesRef minTerm = null; + BytesRef maxTerm = new BytesRef(); + while (true) { + BytesRef term = termsEnum.next(); + if (term == null) { + break; + } + if (minTerm == null) { + minTerm = BytesRef.deepCopyOf(term); + } + maxTerm.copyBytes(term); + termsWriter.write(term, termsEnum); + } + + termsWriter.finish(minTerm, minTerm == null ? null : maxTerm); + } + } + + static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) { + assert fp < (1L << 62); + return (fp << 2) | (hasTerms ? OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? 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 BytesRef term; + // stats + metadata + public final BlockTermState state; + + public PendingTerm(BytesRef term, BlockTermState state) { + super(true); + this.term = term; + this.state = state; + } + + @Override + public String toString() { + return term.utf8ToString(); + } + } + + 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; + private final IntsRef scratchIntsRef = new IntsRef(); + /** Max version for all terms in this block. */ + private final long maxVersion; + + public PendingBlock(BytesRef prefix, long maxVersion, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List>> subIndices) { + super(false); + this.prefix = prefix; + this.maxVersion = maxVersion; + this.fp = fp; + this.hasTerms = hasTerms; + this.isFloor = isFloor; + this.floorLeadByte = floorLeadByte; + this.subIndices = subIndices; + } + + @Override + public String toString() { + return "BLOCK: " + prefix.utf8ToString(); + } + + public void compileIndex(List floorBlocks, RAMOutputStream scratchBytes) throws IOException { + + assert (isFloor && floorBlocks != null && floorBlocks.size() != 0) || (!isFloor && floorBlocks == null): "isFloor=" + isFloor + " floorBlocks=" + floorBlocks; + + assert scratchBytes.getFilePointer() == 0; + + long maxVersionIndex = maxVersion; + + // 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(floorBlocks.size()); + for (PendingBlock sub : floorBlocks) { + assert sub.floorLeadByte != -1; + maxVersionIndex = Math.max(maxVersionIndex, sub.maxVersion); + //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 Builder> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, + 0, 0, true, false, Integer.MAX_VALUE, + FST_OUTPUTS, null, false, + PackedInts.COMPACT, true, 15); + //if (DEBUG) { + // System.out.println(" compile index for prefix=" + prefix); + //} + //indexBuilder.DEBUG = false; + final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()]; + assert bytes.length > 0; + scratchBytes.writeTo(bytes, 0); + indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef), FST_OUTPUTS.newPair(new BytesRef(bytes, 0, bytes.length), Long.MAX_VALUE - maxVersionIndex)); + scratchBytes.reset(); + + // Copy over index for all sub-blocks + + if (subIndices != null) { + for(FST> subIndex : subIndices) { + append(indexBuilder, subIndex); + } + } + + if (floorBlocks != null) { + for (PendingBlock sub : floorBlocks) { + if (sub.subIndices != null) { + for(FST> subIndex : sub.subIndices) { + append(indexBuilder, subIndex); + } + } + sub.subIndices = null; + } + } + + index = indexBuilder.finish(); + 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(Builder> builder, FST> subIndex) 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); + //} + builder.add(Util.toIntsRef(indexEnt.input, scratchIntsRef), indexEnt.output); + } + } + } + + final RAMOutputStream scratchBytes = new RAMOutputStream(); + + class TermsWriter { + private final FieldInfo fieldInfo; + private final int longsSize; + private long numTerms; + final FixedBitSet docsSeen; + long indexStartFP; + + // Used only to partition terms into the block tree; we + // don't pull an FST from this builder: + private final NoOutputs noOutputs; + private final Builder blockBuilder; + + // PendingTerm or PendingBlock: + private final List pending = new ArrayList<>(); + + // Index into pending of most recently written block + private int lastBlockIndex = -1; + + // Re-used when segmenting a too-large block into floor + // blocks: + private int[] subBytes = new int[10]; + private int[] subTermCounts = new int[10]; + private int[] subTermCountSums = new int[10]; + private int[] subSubCounts = new int[10]; + + // This class assigns terms to blocks "naturally", ie, + // according to the number of terms under a given prefix + // that we encounter: + private class FindBlocks extends Builder.FreezeTail { + + @Override + public void freeze(final Builder.UnCompiledNode[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException { + + //if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1); + + for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { + final Builder.UnCompiledNode node = frontier[idx]; + + long totCount = 0; + + if (node.isFinal) { + totCount++; + } + + for(int arcIdx=0;arcIdx target = (Builder.UnCompiledNode) node.arcs[arcIdx].target; + totCount += target.inputCount; + target.clear(); + node.arcs[arcIdx].target = null; + } + node.numArcs = 0; + + if (totCount >= minItemsInBlock || idx == 0) { + // We are on a prefix node that has enough + // entries (terms or sub-blocks) under it to let + // us write a new block or multiple blocks (main + // block + follow on floor blocks): + //if (DEBUG) { + // if (totCount < minItemsInBlock && idx != 0) { + // System.out.println(" force block has terms"); + // } + //} + writeBlocks(lastInput, idx, (int) totCount); + node.inputCount = 1; + } else { + // stragglers! carry count upwards + node.inputCount = totCount; + } + frontier[idx] = new Builder.UnCompiledNode<>(blockBuilder, idx); + } + } + } + + // Write the top count entries on the pending stack as + // one or more blocks. Returns how many blocks were + // written. If the entry count is <= maxItemsPerBlock + // we just write a single block; else we break into + // primary (initial) block and then one or more + // following floor blocks: + + void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException { + if (count <= maxItemsInBlock) { + // Easy case: not floor block. Eg, prefix is "foo", + // and we found 30 terms/sub-blocks starting w/ that + // prefix, and minItemsInBlock <= 30 <= + // maxItemsInBlock. + final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true); + nonFloorBlock.compileIndex(null, scratchBytes); + pending.add(nonFloorBlock); + } else { + // Floor block case. Eg, prefix is "foo" but we + // have 100 terms/sub-blocks starting w/ that + // prefix. We segment the entries into a primary + // block and following floor blocks using the first + // label in the suffix to assign to floor blocks. + + // TODO: we could store min & max suffix start byte + // in each block, to make floor blocks authoritative + + /* + if (DEBUG) { + final BytesRef prefix = new BytesRef(prefixLength); + for(int m=0;m= minItemsInBlock) { + final int curPrefixLength; + if (startLabel == -1) { + curPrefixLength = prefixLength; + } else { + curPrefixLength = 1+prefixLength; + // floor term: + prevTerm.ints[prevTerm.offset + prefixLength] = startLabel; + } + //System.out.println(" " + subCount + " subs"); + final PendingBlock floorBlock = writeBlock(prevTerm, prefixLength, curPrefixLength, curStart, pendingCount, subTermCountSums[1+sub], true, startLabel, curStart == pendingCount); + if (firstBlock == null) { + firstBlock = floorBlock; + } else { + floorBlocks.add(floorBlock); + } + curStart -= pendingCount; + //System.out.println(" = " + pendingCount); + pendingCount = 0; + + assert minItemsInBlock == 1 || subCount > 1: "minItemsInBlock=" + minItemsInBlock + " subCount=" + subCount + " sub=" + sub + " of " + numSubs + " subTermCount=" + subTermCountSums[sub] + " subSubCount=" + subSubCounts[sub] + " depth=" + prefixLength; + subCount = 0; + startLabel = subBytes[sub+1]; + + if (curStart == 0) { + break; + } + + if (curStart <= maxItemsInBlock) { + // remainder is small enough to fit into a + // block. NOTE that this may be too small (< + // minItemsInBlock); need a true segmenter + // here + assert startLabel != -1; + assert firstBlock != null; + prevTerm.ints[prevTerm.offset + prefixLength] = startLabel; + //System.out.println(" final " + (numSubs-sub-1) + " subs"); + /* + for(sub++;sub < numSubs;sub++) { + System.out.println(" " + (subTermCounts[sub] + subSubCounts[sub])); + } + System.out.println(" = " + curStart); + if (curStart < minItemsInBlock) { + System.out.println(" **"); + } + */ + floorBlocks.add(writeBlock(prevTerm, prefixLength, prefixLength+1, curStart, curStart, 0, true, startLabel, true)); + break; + } + } + } + + prevTerm.ints[prevTerm.offset + prefixLength] = savLabel; + + assert firstBlock != null; + firstBlock.compileIndex(floorBlocks, scratchBytes); + + pending.add(firstBlock); + //if (DEBUG) System.out.println(" done pending.size()=" + pending.size()); + } + lastBlockIndex = pending.size()-1; + } + + // for debugging + @SuppressWarnings("unused") + private String toString(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(); + } + } + + // Writes all entries in the pending slice as a single + // block: + private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length, + int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException { + + assert length > 0; + + final int start = pending.size()-startBackwards; + + assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length; + + final List slice = pending.subList(start, start + length); + + final long startFP = out.getFilePointer(); + + final BytesRef prefix = new BytesRef(indexPrefixLength); + for(int m=0;m= 0; + metaWriter.writeVLong(longs[pos]); + } + bytesWriter.writeTo(metaWriter); + bytesWriter.reset(); + absolute = false; + + termCount++; + } else { + PendingBlock block = (PendingBlock) ent; + maxVersionInBlock = Math.max(maxVersionInBlock, block.maxVersion); + final int suffix = block.prefix.length - prefixLength; + + assert suffix > 0; + + // For non-leaf block we borrow 1 bit to record + // if entry is term or sub-block + suffixWriter.writeVInt((suffix<<1)|1); + suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix); + assert block.fp < startFP; + + // if (DEBUG) { + // BytesRef suffixBytes = new BytesRef(suffix); + // System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix); + // suffixBytes.length = suffix; + // System.out.println(" " + (countx++) + ": write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor); + // } + + suffixWriter.writeVLong(startFP - block.fp); + subIndices.add(block.index); + } + } + + assert subIndices.size() != 0; + } + + // TODO: we could block-write the term suffix pointers; + // this would take more space but would enable binary + // search on lookup + + // Write suffixes byte[] blob to terms dict output: + out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0)); + suffixWriter.writeTo(out); + suffixWriter.reset(); + + // Write term meta data byte[] blob + out.writeVInt((int) metaWriter.getFilePointer()); + metaWriter.writeTo(out); + metaWriter.reset(); + + // Remove slice replaced by block: + slice.clear(); + + if (lastBlockIndex >= start) { + if (lastBlockIndex < start+length) { + lastBlockIndex = start; + } else { + lastBlockIndex -= length; + } + } + + // if (DEBUG) { + // System.out.println(" fpEnd=" + out.getFilePointer()); + // } + + return new PendingBlock(prefix, maxVersionInBlock, startFP, termCount != 0, isFloor, floorLeadByte, subIndices); + } + + TermsWriter(FieldInfo fieldInfo) { + this.fieldInfo = fieldInfo; + docsSeen = new FixedBitSet(maxDoc); + + noOutputs = NoOutputs.getSingleton(); + + // This Builder is just used transiently to fragment + // terms into "good" blocks; we don't save the + // resulting FST: + blockBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, + 0, 0, true, + true, Integer.MAX_VALUE, + noOutputs, + new FindBlocks(), false, + PackedInts.COMPACT, + true, 15); + + this.longsSize = postingsWriter.setField(fieldInfo); + } + + private final IntsRef scratchIntsRef = new IntsRef(); + + /** Writes one term's worth of postings. */ + public void write(BytesRef text, TermsEnum termsEnum) throws IOException { + + BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen); + // TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush: + if (state != null && ((IDVersionPostingsWriter) postingsWriter).lastDocID != -1) { + assert state.docFreq != 0; + assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter; + blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput()); + + PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state); + pending.add(term); + numTerms++; + } + } + + // Finishes all terms in this field + public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException { + if (numTerms > 0) { + blockBuilder.finish(); + + // 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; + assert root.index.getEmptyOutput() != null; + + // Write FST to index + indexStartFP = indexOut.getFilePointer(); + root.index.save(indexOut); + //System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name); + + // if (SAVE_DOT_FILES || 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(); + // } + + fields.add(new FieldMetaData(fieldInfo, + ((PendingBlock) pending.get(0)).index.getEmptyOutput(), + numTerms, + indexStartFP, + longsSize, + minTerm, maxTerm)); + } else { + // cannot assert this: we skip deleted docIDs in the postings: + // assert docsSeen.cardinality() == 0; + } + } + + private final RAMOutputStream suffixWriter = new RAMOutputStream(); + private final RAMOutputStream metaWriter = new RAMOutputStream(); + private final RAMOutputStream bytesWriter = new RAMOutputStream(); + } + + @Override + public void close() throws IOException { + + boolean success = false; + try { + + final long dirStart = out.getFilePointer(); + final long indexDirStart = indexOut.getFilePointer(); + + out.writeVInt(fields.size()); + + for(FieldMetaData field : fields) { + //System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms"); + out.writeVInt(field.fieldInfo.number); + assert field.numTerms > 0; + out.writeVLong(field.numTerms); + out.writeVInt(field.rootCode.output1.length); + out.writeBytes(field.rootCode.output1.bytes, field.rootCode.output1.offset, field.rootCode.output1.length); + out.writeVLong(field.rootCode.output2); + out.writeVInt(field.longsSize); + indexOut.writeVLong(field.indexStartFP); + writeBytesRef(out, field.minTerm); + writeBytesRef(out, field.maxTerm); + } + writeTrailer(out, dirStart); + CodecUtil.writeFooter(out); + writeIndexTrailer(indexOut, indexDirStart); + CodecUtil.writeFooter(indexOut); + success = true; + } finally { + if (success) { + IOUtils.close(out, indexOut, postingsWriter); + } else { + IOUtils.closeWhileHandlingException(out, indexOut, postingsWriter); + } + } + } + + private static void writeBytesRef(IndexOutput out, BytesRef bytes) throws IOException { + out.writeVInt(bytes.length); + out.writeBytes(bytes.bytes, bytes.offset, bytes.length); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionFieldReader.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionFieldReader.java new file mode 100644 index 00000000000..417ce11b737 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionFieldReader.java @@ -0,0 +1,163 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.FieldInfo; +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.BytesRef; +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.PairOutputs.Pair; + +/** BlockTree's implementation of {@link Terms}. */ +// public for CheckIndex: +final class VersionFieldReader extends Terms { + final long numTerms; + final FieldInfo fieldInfo; + final long sumTotalTermFreq; + final long sumDocFreq; + final int docCount; + final long indexStartFP; + final long rootBlockFP; + final Pair rootCode; + final BytesRef minTerm; + final BytesRef maxTerm; + final int longsSize; + final VersionBlockTreeTermsReader parent; + + final FST> index; + //private boolean DEBUG; + + VersionFieldReader(VersionBlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, Pair rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, + long indexStartFP, int longsSize, 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.indexStartFP = indexStartFP; + this.rootCode = rootCode; + this.longsSize = longsSize; + 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.output1.bytes, rootCode.output1.offset, rootCode.output1.length)).readVLong() >>> VersionBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS; + + if (indexIn != null) { + final IndexInput clone = indexIn.clone(); + //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name); + clone.seek(indexStartFP); + index = new FST<>(clone, VersionBlockTreeTermsWriter.FST_OUTPUTS); + + /* + 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(); + } + */ + } else { + index = null; + } + } + + @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; + } + } + + @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(TermsEnum reuse) throws IOException { + return new IDVersionSegmentTermsEnum(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; + } + + /** Returns approximate RAM bytes used */ + public long ramBytesUsed() { + return ((index!=null)? index.sizeInBytes() : 0); + } +} diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/package.html b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/package.html new file mode 100644 index 00000000000..16f92efdcf5 --- /dev/null +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/package.html @@ -0,0 +1,26 @@ + + + + + + + +A primary-key postings format that associates a version (long) with each term and +can provide fail-fast lookups by ID and version. + + diff --git a/lucene/sandbox/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/sandbox/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat new file mode 100644 index 00000000000..a319f24b971 --- /dev/null +++ b/lucene/sandbox/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -0,0 +1,16 @@ +# 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. + +org.apache.lucene.codecs.idversion.IDVersionPostingsFormat diff --git a/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/StringAndPayloadField.java b/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/StringAndPayloadField.java new file mode 100644 index 00000000000..19ff31a89d8 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/StringAndPayloadField.java @@ -0,0 +1,104 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.util.BytesRef; + +// TODO: can we take a BytesRef token instead? + +/** Produces a single String token from the provided value, with the provided payload. */ +class StringAndPayloadField extends Field { + + public static final FieldType TYPE = new FieldType(); + + static { + TYPE.setIndexed(true); + TYPE.setOmitNorms(true); + TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS); + TYPE.setTokenized(true); + TYPE.freeze(); + } + + private final BytesRef payload; + + public StringAndPayloadField(String name, String value, BytesRef payload) { + super(name, value, TYPE); + this.payload = payload; + } + + @Override + public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) throws IOException { + SingleTokenWithPayloadTokenStream ts; + if (reuse instanceof SingleTokenWithPayloadTokenStream) { + ts = (SingleTokenWithPayloadTokenStream) reuse; + } else { + ts = new SingleTokenWithPayloadTokenStream(); + } + ts.setValue((String) fieldsData, payload); + return ts; + } + + static final class SingleTokenWithPayloadTokenStream extends TokenStream { + + private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class); + private final PayloadAttribute payloadAttribute = addAttribute(PayloadAttribute.class); + private boolean used = false; + private String value = null; + private BytesRef payload; + + /** Sets the string value. */ + void setValue(String value, BytesRef payload) { + this.value = value; + this.payload = payload; + } + + @Override + public boolean incrementToken() { + if (used) { + return false; + } + clearAttributes(); + termAttribute.append(value); + payloadAttribute.setPayload(payload); + used = true; + return true; + } + + @Override + public void reset() { + used = false; + } + + @Override + public void close() { + value = null; + payload = null; + } + } +} + + diff --git a/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java b/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java new file mode 100644 index 00000000000..9ff8acef959 --- /dev/null +++ b/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java @@ -0,0 +1,841 @@ +package org.apache.lucene.codecs.idversion; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.analysis.Analyzer.TokenStreamComponents; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockTokenFilter; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.codecs.idversion.StringAndPayloadField.SingleTokenWithPayloadTokenStream; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.index.ConcurrentMergeScheduler; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MergeScheduler; +import org.apache.lucene.index.PerThreadPKLookup; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.TieredMergePolicy; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LiveFieldValues; +import org.apache.lucene.search.SearcherFactory; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; + +/** + * Basic tests for IDVersionPostingsFormat + */ +// Cannot extend BasePostingsFormatTestCase because this PF is not +// general (it requires payloads, only allows 1 doc per term, etc.) +public class TestIDVersionPostingsFormat extends LuceneTestCase { + + public void testBasic() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())); + iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat())); + RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); + Document doc = new Document(); + doc.add(makeIDField("id0", 100)); + w.addDocument(doc); + doc = new Document(); + doc.add(makeIDField("id1", 110)); + w.addDocument(doc); + IndexReader r = w.getReader(); + IDVersionSegmentTermsEnum termsEnum = (IDVersionSegmentTermsEnum) r.leaves().get(0).reader().fields().terms("id").iterator(null); + assertTrue(termsEnum.seekExact(new BytesRef("id0"), 50)); + assertTrue(termsEnum.seekExact(new BytesRef("id0"), 100)); + assertFalse(termsEnum.seekExact(new BytesRef("id0"), 101)); + assertTrue(termsEnum.seekExact(new BytesRef("id1"), 50)); + assertTrue(termsEnum.seekExact(new BytesRef("id1"), 110)); + assertFalse(termsEnum.seekExact(new BytesRef("id1"), 111)); + r.close(); + + w.close(); + dir.close(); + } + + private interface IDSource { + String next(); + } + + private IDSource getRandomIDs() { + IDSource ids; + switch (random().nextInt(6)) { + case 0: + // random simple + if (VERBOSE) { + System.out.println("TEST: use random simple ids"); + } + ids = new IDSource() { + @Override + public String next() { + return TestUtil.randomSimpleString(random()); + } + }; + break; + case 1: + // random realistic unicode + if (VERBOSE) { + System.out.println("TEST: use random realistic unicode ids"); + } + ids = new IDSource() { + @Override + public String next() { + return TestUtil.randomRealisticUnicodeString(random()); + } + }; + break; + case 2: + // sequential + if (VERBOSE) { + System.out.println("TEST: use seuquential ids"); + } + ids = new IDSource() { + int upto; + @Override + public String next() { + return Integer.toString(upto++); + } + }; + break; + case 3: + // zero-pad sequential + if (VERBOSE) { + System.out.println("TEST: use zero-pad seuquential ids"); + } + ids = new IDSource() { + final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX); + final String zeroPad = String.format(Locale.ROOT, "%0" + TestUtil.nextInt(random(), 4, 20) + "d", 0); + int upto; + @Override + public String next() { + String s = Integer.toString(upto++); + return zeroPad.substring(zeroPad.length() - s.length()) + s; + } + }; + break; + case 4: + // random long + if (VERBOSE) { + System.out.println("TEST: use random long ids"); + } + ids = new IDSource() { + final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX); + int upto; + @Override + public String next() { + return Long.toString(random().nextLong() & 0x3ffffffffffffffL, radix); + } + }; + break; + case 5: + // zero-pad random long + if (VERBOSE) { + System.out.println("TEST: use zero-pad random long ids"); + } + ids = new IDSource() { + final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX); + final String zeroPad = String.format(Locale.ROOT, "%015d", 0); + int upto; + @Override + public String next() { + return Long.toString(random().nextLong() & 0x3ffffffffffffffL, radix); + } + }; + break; + default: + throw new AssertionError(); + } + + return ids; + } + + // TODO make a similar test for BT, w/ varied IDs: + + public void testRandom() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())); + int minItemsInBlock = TestUtil.nextInt(random(), 2, 50); + int maxItemsInBlock = 2*(minItemsInBlock-1) + random().nextInt(50); + iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat(minItemsInBlock, maxItemsInBlock))); + RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); + //IndexWriter w = new IndexWriter(dir, iwc); + int numDocs = atLeast(1000); + Map idValues = new HashMap(); + int docUpto = 0; + if (VERBOSE) { + System.out.println("TEST: numDocs=" + numDocs); + } + + IDSource ids = getRandomIDs(); + String idPrefix; + if (random().nextBoolean()) { + idPrefix = ""; + } else { + idPrefix = TestUtil.randomSimpleString(random()); + if (VERBOSE) { + System.out.println("TEST: use id prefix: " + idPrefix); + } + } + + boolean useMonotonicVersion = random().nextBoolean(); + if (VERBOSE) { + System.out.println("TEST: useMonotonicVersion=" + useMonotonicVersion); + } + + List idsList = new ArrayList<>(); + + long version = 0; + while (docUpto < numDocs) { + String idValue = idPrefix + ids.next(); + if (idValues.containsKey(idValue)) { + continue; + } + if (useMonotonicVersion) { + version += TestUtil.nextInt(random(), 1, 10); + } else { + version = random().nextLong() & 0x3fffffffffffffffL; + } + idValues.put(idValue, version); + if (VERBOSE) { + System.out.println(" " + idValue + " -> " + version); + } + Document doc = new Document(); + doc.add(makeIDField(idValue, version)); + w.addDocument(doc); + idsList.add(idValue); + + if (idsList.size() > 0 && random().nextInt(7) == 5) { + // Randomly delete or update a previous ID + idValue = idsList.get(random().nextInt(idsList.size())); + if (random().nextBoolean()) { + if (useMonotonicVersion) { + version += TestUtil.nextInt(random(), 1, 10); + } else { + version = random().nextLong() & 0x3fffffffffffffffL; + } + doc = new Document(); + doc.add(makeIDField(idValue, version)); + if (VERBOSE) { + System.out.println(" update " + idValue + " -> " + version); + } + w.updateDocument(new Term("id", idValue), doc); + idValues.put(idValue, version); + } else { + if (VERBOSE) { + System.out.println(" delete " + idValue); + } + w.deleteDocuments(new Term("id", idValue)); + idValues.remove(idValue); + } + } + + docUpto++; + } + + IndexReader r = w.getReader(); + //IndexReader r = DirectoryReader.open(w, true); + PerThreadVersionPKLookup lookup = new PerThreadVersionPKLookup(r, "id"); + + List> idValuesList = new ArrayList<>(idValues.entrySet()); + int iters = numDocs * 5; + for(int iter=0;iter idsSeen = new HashSet(); + while (idsSeen.size() < numIDs) { + idsSeen.add(idsSource.next()); + } + final String[] ids = idsSeen.toArray(new String[numIDs]); + + final Object[] locks = new Object[ids.length]; + for(int i=0;i versionValues = new LiveFieldValues(mgr, missingValue) { + @Override + protected Long lookupFromSearcher(IndexSearcher s, String id) { + // TODO: would be cleaner if we could do our PerThreadLookup here instead of "up above": + // We always return missing: the caller then does a lookup against the current reader + return missingValue; + } + }; + + // Maps to the version the id was lasted indexed with: + final Map truth = new ConcurrentHashMap<>(); + + final CountDownLatch startingGun = new CountDownLatch(1); + + Thread[] threads = new Thread[TestUtil.nextInt(random(), 2, 7)]; + + final int versionType = random().nextInt(3); + + if (VERBOSE) { + if (versionType == 0) { + System.out.println("TEST: use random versions"); + } else if (versionType == 1) { + System.out.println("TEST: use monotonic versions"); + } else { + System.out.println("TEST: use nanotime versions"); + } + } + + // Run for 3 sec in normal tests, else 60 seconds for nightly: + final long stopTime = System.currentTimeMillis() + (TEST_NIGHTLY ? 60000 : 3000); + + for(int i=0;i= newVersion); + } + } finally { + mgr.release(s); + } + } + } + } + }; + threads[i].start(); + } + + startingGun.countDown(); + + // Keep reopening the NRT reader until all indexing threads are done: + refreshLoop: + while (true) { + Thread.sleep(TestUtil.nextInt(random(), 1, 10)); + mgr.maybeRefresh(); + for (Thread thread : threads) { + if (thread.isAlive()) { + continue refreshLoop; + } + } + + break; + } + + // Verify final index against truth: + for(int i=0;i<2;i++) { + mgr.maybeRefresh(); + IndexSearcher s = mgr.acquire(); + try { + IndexReader r = s.getIndexReader(); + // cannot assert this: maybe not all IDs were indexed + /* + assertEquals(numIDs, r.numDocs()); + if (i == 1) { + // After forceMerge no deleted docs: + assertEquals(numIDs, r.maxDoc()); + } + */ + PerThreadVersionPKLookup lookup = new PerThreadVersionPKLookup(r, "id"); + for(Map.Entry ent : truth.entrySet()) { + assertTrue(lookup.lookup(new BytesRef(ent.getKey()), -1L) != -1); + assertEquals(ent.getValue().longValue(), lookup.getVersion()); + } + } finally { + mgr.release(s); + } + + if (i == 1) { + break; + } + + // forceMerge and verify again + w.forceMerge(1); + } + + mgr.close(); + w.close(); + dir.close(); + } +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java index 01dea239a08..32437443046 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java @@ -19,9 +19,9 @@ package org.apache.lucene.codecs.lucene40; import java.io.IOException; -import org.apache.lucene.codecs.BlockTreeTermsWriter; 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.LuceneTestCase; 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 33835ad336a..7038ae912af 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 @@ -20,8 +20,6 @@ package org.apache.lucene.codecs.mockrandom; import java.io.IOException; import java.util.Random; -import org.apache.lucene.codecs.BlockTreeTermsReader; -import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; @@ -36,6 +34,8 @@ 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.lucene41.Lucene41PostingsReader; import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter; import org.apache.lucene.codecs.memory.FSTOrdTermsReader; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java index 64b40158b30..0c851407253 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java @@ -19,13 +19,13 @@ package org.apache.lucene.codecs.nestedpulsing; import java.io.IOException; -import org.apache.lucene.codecs.BlockTreeTermsReader; -import org.apache.lucene.codecs.BlockTreeTermsWriter; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader; import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter; import org.apache.lucene.codecs.pulsing.PulsingPostingsReader; diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java index b6d7c2ad1c2..7eae29441b7 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java @@ -6,6 +6,7 @@ import java.util.Iterator; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.VirtualMethod; import org.apache.lucene.util.automaton.CompiledAutomaton; /* @@ -117,12 +118,16 @@ public class AssertingAtomicReader extends FilterAtomicReader { } } + static final VirtualMethod SEEK_EXACT = new VirtualMethod<>(TermsEnum.class, "seekExact", BytesRef.class); + static class AssertingTermsEnum extends FilterTermsEnum { private enum State {INITIAL, POSITIONED, UNPOSITIONED}; private State state = State.INITIAL; + private final boolean delegateOverridesSeekExact; public AssertingTermsEnum(TermsEnum in) { super(in); + delegateOverridesSeekExact = SEEK_EXACT.isOverriddenAsOf(in.getClass()); } @Override @@ -213,13 +218,18 @@ public class AssertingAtomicReader extends FilterAtomicReader { @Override public boolean seekExact(BytesRef text) throws IOException { assert text.isValid(); - if (super.seekExact(text)) { + boolean result; + if (delegateOverridesSeekExact) { + result = in.seekExact(text); + } else { + result = super.seekExact(text); + } + if (result) { state = State.POSITIONED; - return true; } else { state = State.UNPOSITIONED; - return false; } + return result; } @Override @@ -234,6 +244,11 @@ public class AssertingAtomicReader extends FilterAtomicReader { super.seekExact(term, state); this.state = State.POSITIONED; } + + @Override + public String toString() { + return "AssertingTermsEnum(" + in + ")"; + } } static enum DocsEnumState { START, ITERATING, FINISHED }; @@ -682,4 +697,4 @@ public class AssertingAtomicReader extends FilterAtomicReader { } private final Object cacheKey = new Object(); -} \ No newline at end of file +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java b/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java new file mode 100644 index 00000000000..8e6dd51e5f7 --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java @@ -0,0 +1,97 @@ +package org.apache.lucene.index; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; + +/** Utility class to do efficient primary-key (only 1 doc contains the + * given term) lookups by segment, re-using the enums. This class is + * not thread safe, so it is the caller's job to create and use one + * instance of this per thread. Do not use this if a term may appear + * in more than one document! It will only return the first one it + * finds. */ +public class PerThreadPKLookup { + + protected final TermsEnum[] termsEnums; + protected final DocsEnum[] docsEnums; + protected final Bits[] liveDocs; + protected final int[] docBases; + protected final int numSegs; + protected final boolean hasDeletions; + + public PerThreadPKLookup(IndexReader r, String idFieldName) throws IOException { + + List leaves = new ArrayList<>(r.leaves()); + + // Larger segments are more likely to have the id, so we sort largest to smallest by numDocs: + Collections.sort(leaves, new Comparator() { + @Override + public int compare(AtomicReaderContext c1, AtomicReaderContext c2) { + return c2.reader().numDocs() - c1.reader().numDocs(); + } + }); + + termsEnums = new TermsEnum[leaves.size()]; + docsEnums = new DocsEnum[leaves.size()]; + liveDocs = new Bits[leaves.size()]; + docBases = new int[leaves.size()]; + int numSegs = 0; + boolean hasDeletions = false; + for(int i=0;i SCORE_COLLECTOR = new VirtualMethod(BulkScorer.class, "score", LeafCollector.class); - private static final VirtualMethod SCORE_COLLECTOR_RANGE = new VirtualMethod(BulkScorer.class, "score", LeafCollector.class, int.class); + private static final VirtualMethod SCORE_COLLECTOR = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class); + private static final VirtualMethod SCORE_COLLECTOR_RANGE = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class, int.class); public static BulkScorer wrap(Random random, BulkScorer other) { if (other == null || other instanceof AssertingBulkScorer) {