diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsReaderBase.java b/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsReaderBase.java
new file mode 100644
index 00000000000..690b744e550
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsReaderBase.java
@@ -0,0 +1,77 @@
+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.IOException;
+import java.io.Closeable;
+
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.codecs.temp.TempTermState;
+
+/** The core terms dictionaries (BlockTermsReader,
+ * BlockTreeTermsReader) interact with a single instance
+ * of this class to manage creation of {@link DocsEnum} and
+ * {@link DocsAndPositionsEnum} instances. It provides an
+ * IndexInput (termsIn) where this class may read any
+ * previously stored data that it had written in its
+ * corresponding {@link PostingsWriterBase} at indexing
+ * time.
+ * @lucene.experimental */
+
+// 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
+public abstract class TempPostingsReaderBase implements Closeable {
+
+ /** Sole constructor. (For invocation by subclass
+ * constructors, typically implicit.) */
+ protected TempPostingsReaderBase() {
+ }
+
+ /** Performs any initialization, such as reading and
+ * verifying the header from the provided terms
+ * dictionary {@link IndexInput}. */
+ public abstract void init(IndexInput termsIn) throws IOException;
+
+ /** Return a newly created empty TermState */
+ public abstract TempTermState newTermState() throws IOException;
+
+ /** Actually decode metadata for next term */
+ public abstract void nextTerm(FieldInfo fieldInfo, TempTermState state) throws IOException;
+
+ /** Must fully consume state, since after this call that
+ * TermState may be reused. */
+ public abstract DocsEnum docs(FieldInfo fieldInfo, TempTermState state, Bits skipDocs, DocsEnum reuse, int flags) throws IOException;
+
+ /** Must fully consume state, since after this call that
+ * TermState may be reused. */
+ public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TempTermState state, Bits skipDocs, DocsAndPositionsEnum reuse,
+ int flags) throws IOException;
+
+ @Override
+ public abstract void close() throws IOException;
+
+ /** Reads data for all terms in the next block; this
+ * method should merely load the byte[] blob but not
+ * decode, which is done in {@link #nextTerm}. */
+ public abstract void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, TempTermState termState) throws IOException;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsWriterBase.java b/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsWriterBase.java
new file mode 100644
index 00000000000..ef41ea3a024
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TempPostingsWriterBase.java
@@ -0,0 +1,73 @@
+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.IOException;
+import java.io.Closeable;
+
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.index.FieldInfo;
+
+/**
+ * Extension of {@link PostingsConsumer} to support pluggable term dictionaries.
+ *
+ * This class contains additional hooks to interact with the provided
+ * term dictionaries such as {@link BlockTreeTermsWriter}. If you want
+ * to re-use an existing implementation and are only interested in
+ * customizing the format of the postings list, extend this class
+ * instead.
+ *
+ * @see PostingsReaderBase
+ * @lucene.experimental
+ */
+// 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
+public abstract class TempPostingsWriterBase extends PostingsConsumer implements Closeable {
+
+ /** Sole constructor. (For invocation by subclass
+ * constructors, typically implicit.) */
+ protected TempPostingsWriterBase() {
+ }
+
+ /** Called once after startup, before any terms have been
+ * added. Implementations typically write a header to
+ * the provided {@code termsOut}. */
+ public abstract void start(IndexOutput termsOut) throws IOException;
+
+ /** Start a new term. Note that a matching call to {@link
+ * #finishTerm(TermStats)} is done, only if the term has at least one
+ * document. */
+ public abstract void startTerm() throws IOException;
+
+ /** Flush count terms starting at start "backwards", as a
+ * block. start is a negative offset from the end of the
+ * terms stack, ie bigger start means further back in
+ * the stack. */
+ public abstract void flushTermsBlock(int start, int count) throws IOException;
+
+ /** Finishes the current term. The provided {@link
+ * TermStats} contains the term's summary statistics. */
+ public abstract void finishTerm(TermStats stats) throws IOException;
+
+ /** Called when the writing switches to another field. */
+ public abstract void setField(FieldInfo fieldInfo);
+
+ @Override
+ public abstract void close() throws IOException;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
index 64178590c57..30490dfa339 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
@@ -33,7 +33,7 @@ import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZ
* Encode all values in normal area with fixed bit width,
* which is determined by the max value in this block.
*/
-final class ForUtil {
+public final class ForUtil {
/**
* Special number of bits per value used whenever all values to encode are equal.
@@ -44,7 +44,7 @@ final class ForUtil {
* Upper limit of the number of bytes that might be required to stored
* BLOCK_SIZE encoded values.
*/
- static final int MAX_ENCODED_SIZE = BLOCK_SIZE * 4;
+ public static final int MAX_ENCODED_SIZE = BLOCK_SIZE * 4;
/**
* Upper limit of the number of values that might be decoded in a single call to
@@ -52,7 +52,7 @@ final class ForUtil {
* BLOCK_SIZE are garbage, it is necessary to allocate value buffers
* whose size is >= MAX_DATA_SIZE to avoid {@link ArrayIndexOutOfBoundsException}s.
*/
- static final int MAX_DATA_SIZE;
+ public static final int MAX_DATA_SIZE;
static {
int maxDataSize = 0;
for(int version=PackedInts.VERSION_START;version<=PackedInts.VERSION_CURRENT;version++) {
@@ -96,7 +96,7 @@ final class ForUtil {
/**
* Create a new {@link ForUtil} instance and save state into out.
*/
- ForUtil(float acceptableOverheadRatio, DataOutput out) throws IOException {
+ public ForUtil(float acceptableOverheadRatio, DataOutput out) throws IOException {
out.writeVInt(PackedInts.VERSION_CURRENT);
encodedSizes = new int[33];
encoders = new PackedInts.Encoder[33];
@@ -122,7 +122,7 @@ final class ForUtil {
/**
* Restore a {@link ForUtil} from a {@link DataInput}.
*/
- ForUtil(DataInput in) throws IOException {
+ public ForUtil(DataInput in) throws IOException {
int packedIntsVersion = in.readVInt();
PackedInts.checkVersion(packedIntsVersion);
encodedSizes = new int[33];
@@ -154,7 +154,7 @@ final class ForUtil {
* @param out the destination output
* @throws IOException If there is a low-level I/O error
*/
- void writeBlock(int[] data, byte[] encoded, IndexOutput out) throws IOException {
+ public void writeBlock(int[] data, byte[] encoded, IndexOutput out) throws IOException {
if (isAllEqual(data)) {
out.writeByte((byte) ALL_VALUES_EQUAL);
out.writeVInt(data[0]);
@@ -183,7 +183,7 @@ final class ForUtil {
* @param decoded where to write decoded data
* @throws IOException If there is a low-level I/O error
*/
- void readBlock(IndexInput in, byte[] encoded, int[] decoded) throws IOException {
+ public void readBlock(IndexInput in, byte[] encoded, int[] decoded) throws IOException {
final int numBits = in.readByte();
assert numBits <= 32 : numBits;
@@ -209,7 +209,7 @@ final class ForUtil {
* @param in the input where to read data
* @throws IOException If there is a low-level I/O error
*/
- void skipBlock(IndexInput in) throws IOException {
+ public void skipBlock(IndexInput in) throws IOException {
final int numBits = in.readByte();
if (numBits == ALL_VALUES_EQUAL) {
in.readVInt();
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 df7fe6f65be..4838f7a8625 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
@@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
SkipFPDelta determines the position of this term's SkipData within the .doc
* file. In particular, it is the length of the TermFreq data.
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
- * (i.e. 128 in Lucene41PostingsFormat).
+ * (i.e. 8 in Lucene41PostingsFormat).
*
SingletonDocID is an optimization when a term only appears in one document. In this case, instead
* of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
* single document ID is written to the term dictionary.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
index 483b0ec21df..7dc77fff086 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
@@ -50,7 +50,7 @@ import org.apache.lucene.store.IndexInput;
* Therefore, we'll trim df before passing it to the interface. see trim(int)
*
*/
-final class Lucene41SkipReader extends MultiLevelSkipListReader {
+public final class Lucene41SkipReader extends MultiLevelSkipListReader {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private final int blockSize;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
index 1bd082859d9..22e106218c1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
@@ -43,7 +43,7 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
* 4. start offset.
*
*/
-final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
+public final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private int[] lastSkipDoc;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
index d429cb0fe93..3df02932d1b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
@@ -381,7 +381,13 @@ the term dictionary. Stored fields are compressed by default.
Limitations
-
Lucene uses a Java int to refer to
+
When referring to term numbers, Lucene's current implementation uses a Java
+int to hold the term index, which means the
+maximum number of unique terms in any single index segment is ~2.1 billion
+times the term index interval (default 128) = ~274 billion. This is technically
+not a limitation of the index file format, just of Lucene's current
+implementation.
+
Similarly, Lucene uses a Java int to refer to
document numbers, and the index file format uses an Int32
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermState.java b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermState.java
new file mode 100644
index 00000000000..b6a027d3077
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermState.java
@@ -0,0 +1,47 @@
+package org.apache.lucene.codecs.temp;
+/*
+ * 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 org.apache.lucene.index.TermState;
+
+public class TempBlockTermState extends TempTermState {
+ /** the term's ord in the current block */
+ public int termBlockOrd;
+
+ /** Sole constructor. (For invocation by subclass
+ * constructors, typically implicit.) */
+ protected TempBlockTermState() {
+ }
+
+ public TempBlockTermState clone() {
+ TempBlockTermState other = (TempBlockTermState)super.clone();
+ return other;
+ }
+
+ @Override
+ public void copyFrom(TermState _other) {
+ assert _other instanceof TempBlockTermState : "can not copy from " + _other.getClass().getName();
+ super.copyFrom(_other);
+ TempBlockTermState other = (TempBlockTermState) _other;
+ termBlockOrd = other.termBlockOrd;
+ }
+
+ @Override
+ public String toString() {
+ return super.toString() + " termBlockOrd=" + termBlockOrd;
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermsReader.java
new file mode 100644
index 00000000000..c11afb82c6e
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempBlockTermsReader.java
@@ -0,0 +1,2942 @@
+package org.apache.lucene.codecs.temp;
+
+/*
+ * 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.Comparator;
+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;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
+import org.apache.lucene.codecs.CodecUtil;
+
+/** 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 does not support
+ * index divisor when opening an IndexReader. Instead, you
+ * can change the min/maxItemsPerBlock during indexing.
+ *
+ *
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 TempBlockTermsWriter}.
+ *
+ * @lucene.experimental
+ */
+
+public class TempBlockTermsReader extends FieldsProducer {
+
+ // Open input to the main terms dict file (_X.tib)
+ private final IndexInput in;
+
+ //private static final boolean DEBUG = TempBlockTermsWriter.DEBUG;
+
+ // Reads the terms dict entries, to gather state to
+ // produce DocsEnum on demand
+ private final TempPostingsReaderBase 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 TempBlockTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
+ TempPostingsReaderBase postingsReader, IOContext ioContext,
+ String segmentSuffix, int indexDivisor)
+ throws IOException {
+
+ this.postingsReader = postingsReader;
+
+ this.segment = info.name;
+ in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTermsWriter.TERMS_EXTENSION),
+ ioContext);
+
+ boolean success = false;
+ IndexInput indexIn = null;
+
+ try {
+ version = readHeader(in);
+ if (indexDivisor != -1) {
+ indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTermsWriter.TERMS_INDEX_EXTENSION),
+ ioContext);
+ int indexVersion = readIndexHeader(indexIn);
+ if (indexVersion != version) {
+ throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
+ }
+ }
+
+ // Have PostingsReader init itself
+ postingsReader.init(in);
+
+ // Read per-field details
+ seekDir(in, dirOffset);
+ if (indexDivisor != -1) {
+ 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();
+ 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 = indexDivisor != -1 ? indexIn.readVLong() : 0;
+ FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, indexIn));
+ if (previous != null) {
+ throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
+ }
+ }
+ if (indexDivisor != -1) {
+ indexIn.close();
+ }
+
+ success = true;
+ } finally {
+ if (!success) {
+ // this.close() will close in:
+ IOUtils.closeWhileHandlingException(indexIn, this);
+ }
+ }
+ }
+
+ /** Reads terms file header. */
+ private int readHeader(IndexInput input) throws IOException {
+ int version = CodecUtil.checkHeader(input, TempBlockTermsWriter.TERMS_CODEC_NAME,
+ TempBlockTermsWriter.TERMS_VERSION_START,
+ TempBlockTermsWriter.TERMS_VERSION_CURRENT);
+ if (version < TempBlockTermsWriter.TERMS_VERSION_APPEND_ONLY) {
+ dirOffset = input.readLong();
+ }
+ return version;
+ }
+
+ /** Reads index file header. */
+ private int readIndexHeader(IndexInput input) throws IOException {
+ int version = CodecUtil.checkHeader(input, TempBlockTermsWriter.TERMS_INDEX_CODEC_NAME,
+ TempBlockTermsWriter.TERMS_INDEX_VERSION_START,
+ TempBlockTermsWriter.TERMS_INDEX_VERSION_CURRENT);
+ if (version < TempBlockTermsWriter.TERMS_INDEX_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 >= TempBlockTermsWriter.TERMS_INDEX_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();
+ }
+ }
+ }
+
+ /**
+ * Temp 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, "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();
+
+ /** Temp'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;
+ private final FST index;
+
+ //private boolean DEBUG;
+
+ FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, IndexInput indexIn) throws IOException {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ //DEBUG = TempBlockTermsReader.DEBUG && fieldInfo.name.equals("id");
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.docCount = docCount;
+ this.indexStartFP = indexStartFP;
+ this.rootCode = rootCode;
+ // 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() >>> TempBlockTermsWriter.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;
+ }
+ }
+
+ /** For debugging -- used by CheckIndex too*/
+ // TODO: maybe push this into Terms?
+ public Stats computeStats() throws IOException {
+ return new SegmentTermsEnum().computeBlockStats();
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+ @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);
+ }
+
+ // 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 TempTermState termState;
+
+ // Cumulative output so far
+ BytesRef outputPrefix;
+
+ private int startBytePos;
+ private int suffix;
+
+ public Frame(int ord) throws IOException {
+ this.ord = ord;
+ termState = postingsReader.newTermState();
+ termState.totalTermFreq = -1;
+ }
+
+ 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 & TempBlockTermsWriter.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;
+
+ postingsReader.readTermsBlock(in, fieldInfo, termState);
+
+ 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();
+ assert limit > 0;
+
+ // We must set/incr state.termCount because
+ // postings impl can look at this
+ termState.termBlockOrd = metaDataUpto;
+
+ // 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:
+ 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);
+ }
+
+ postingsReader.nextTerm(fieldInfo, termState);
+ metaDataUpto++;
+ termState.termBlockOrd++;
+ }
+ }
+ }
+
+ 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 = TempBlockTermsReader.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 = TempBlockTermsReader.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];
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+ // 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 >>> TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+ final Frame f = getFrame(1+currentFrame.ord);
+ f.hasTerms = (code & TempBlockTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
+ f.hasTermsOrig = f.hasTerms;
+ f.isFloor = (code & TempBlockTermsWriter.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, final boolean useCache) 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, final boolean useCache) 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, false);
+ 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 TempTermState;
+ 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 TempTermState state;
+
+ public Frame(int ord) throws IOException {
+ this.ord = ord;
+ state = postingsReader.newTermState();
+ state.totalTermFreq = -1;
+ }
+
+ 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
+ postingsReader.readTermsBlock(in, fieldInfo, state);
+
+ // 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();
+ assert limit > 0;
+
+ // We must set/incr state.termCount because
+ // postings impl can look at this
+ state.termBlockOrd = metaDataUpto;
+
+ // 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:
+ 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);
+ }
+
+ postingsReader.nextTerm(fieldInfo, state);
+ metaDataUpto++;
+ state.termBlockOrd++;
+ }
+ }
+
+ // Used only by assert
+ private boolean prefixMatches(BytesRef target) {
+ for(int bytePos=0;bytePos
+ * Writes terms dict and index, block-encoding (column
+ * stride) each term's metadata for each set of terms
+ * between two index terms.
+ *
The .tim file contains the list of terms in each
+ * field along with per-term statistics (such as docfreq)
+ * and per-term metadata (typically pointers to the postings list
+ * for that term in the inverted index).
+ *
+ *
+ *
The .tim is arranged in blocks: with blocks containing
+ * a variable number of entries (by default 25-48), where
+ * each entry is either a term or a reference to a
+ * sub-block.
+ *
+ *
NOTE: The term dictionary can plug into different postings implementations:
+ * the postings writer/reader are actually responsible for encoding
+ * and decoding the Postings Metadata and Term Metadata sections.
Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information
+ * for the BlockTree implementation.
+ *
DirOffset is a pointer to the FieldSummary section.
+ *
DocFreq is the count of documents which contain the term.
+ *
TotalTermFreq is the total number of occurrences of the term. This is encoded
+ * as the difference between the total number of occurrences and the DocFreq.
+ *
FieldNumber is the fields number from {@link FieldInfos}. (.fnm)
+ *
NumTerms is the number of unique terms for the field.
+ *
RootCode points to the root block for the field.
+ *
SumDocFreq is the total number of postings, the number of term-document pairs across
+ * the entire field.
+ *
DocCount is the number of documents that have at least one posting for this field.
+ *
PostingsMetadata and TermMetadata are plugged into by the specific postings implementation:
+ * these contain arbitrary per-file data (such as parameters or versioning information)
+ * and per-term data (such as pointers to inverted files).
+ *
+ *
+ *
Term Index
+ *
The .tip file contains an index into the term dictionary, so that it can be
+ * accessed randomly. The index is also used to determine
+ * when a given term cannot exist on disk (in the .tim file), saving a disk seek.
The .tip file contains a separate FST for each
+ * field. The FST maps a term prefix to the on-disk
+ * block that holds all terms starting with that
+ * prefix. Each field's IndexStartFP points to its
+ * FST.
+ *
DirOffset is a pointer to the start of the IndexStartFPs
+ * for all fields
+ *
It's possible that an on-disk block would contain
+ * too many terms (more than the allowed maximum
+ * (default: 48)). When this happens, the block is
+ * sub-divided into new blocks (called "floor
+ * blocks"), and then the output in the FST for the
+ * block's prefix encodes the leading byte of each
+ * sub-block, and its file pointer.
+ *
+ *
+ * @see BlockTreeTermsReader
+ * @lucene.experimental
+ */
+
+public class TempBlockTermsWriter extends FieldsConsumer {
+
+ /** Suggested default value for the {@code
+ * minItemsInBlock} parameter to {@link
+ * #TempBlockTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int)}. */
+ public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
+
+ /** Suggested default value for the {@code
+ * maxItemsInBlock} parameter to {@link
+ * #TempBlockTermsWriter(SegmentWriteState,TempPostingsWriterBase,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 = "tim";
+ final static String TERMS_CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
+
+ /** Initial terms format. */
+ public static final int TERMS_VERSION_START = 0;
+
+ /** Append-only */
+ public static final int TERMS_VERSION_APPEND_ONLY = 1;
+
+ /** Current terms format. */
+ public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_APPEND_ONLY;
+
+ /** Extension of terms index file */
+ static final String TERMS_INDEX_EXTENSION = "tip";
+ final static String TERMS_INDEX_CODEC_NAME = "BLOCK_TREE_TERMS_INDEX";
+
+ /** Initial index format. */
+ public static final int TERMS_INDEX_VERSION_START = 0;
+
+ /** Append-only */
+ public static final int TERMS_INDEX_VERSION_APPEND_ONLY = 1;
+
+ /** Current index format. */
+ public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_APPEND_ONLY;
+
+ private final IndexOutput out;
+ private final IndexOutput indexOut;
+ final int minItemsInBlock;
+ final int maxItemsInBlock;
+
+ final TempPostingsWriterBase postingsWriter;
+ final FieldInfos fieldInfos;
+ FieldInfo currentField;
+
+ private static class FieldMetaData {
+ public final FieldInfo fieldInfo;
+ public final BytesRef rootCode;
+ public final long numTerms;
+ public final long indexStartFP;
+ public final long sumTotalTermFreq;
+ public final long sumDocFreq;
+ public final int docCount;
+
+ public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount) {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ assert rootCode != null: "field=" + fieldInfo.name + " numTerms=" + numTerms;
+ this.rootCode = rootCode;
+ this.indexStartFP = indexStartFP;
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.docCount = docCount;
+ }
+ }
+
+ 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 TempBlockTermsWriter(
+ SegmentWriteState state,
+ TempPostingsWriterBase 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);
+ }
+
+ 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;
+ writeHeader(out);
+
+ //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);
+ writeIndexHeader(indexOut);
+
+ currentField = null;
+ this.postingsWriter = postingsWriter;
+ // segment = state.segmentName;
+
+ // System.out.println("BTW.init seg=" + state.segmentName);
+
+ postingsWriter.start(out); // have consumer write its format/header
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(out, indexOut);
+ }
+ }
+ this.indexOut = indexOut;
+ }
+
+ /** Writes the terms file header. */
+ private void writeHeader(IndexOutput out) throws IOException {
+ CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
+ }
+
+ /** Writes the index file header. */
+ private void writeIndexHeader(IndexOutput out) throws IOException {
+ CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT);
+ }
+
+ /** 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 TermsConsumer addField(FieldInfo field) throws IOException {
+ //DEBUG = field.name.equals("id");
+ //if (DEBUG) System.out.println("\nBTTW.addField seg=" + segment + " field=" + field.name);
+ assert currentField == null || currentField.name.compareTo(field.name) < 0;
+ currentField = field;
+ return new TermsWriter(field);
+ }
+
+ 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;
+ public final TermStats stats;
+
+ public PendingTerm(BytesRef term, TermStats stats) {
+ super(true);
+ this.term = term;
+ this.stats = stats;
+ }
+
+ @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();
+
+ public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List> subIndices) {
+ super(false);
+ this.prefix = prefix;
+ this.fp = fp;
+ this.hasTerms = hasTerms;
+ this.isFloor = isFloor;
+ this.floorLeadByte = floorLeadByte;
+ this.subIndices = subIndices;
+ }
+
+ @Override
+ public String toString() {
+ return "BLOCK: " + prefix.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;
+
+ // 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;
+ //if (DEBUG) {
+ // System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
+ //}
+ scratchBytes.writeByte((byte) sub.floorLeadByte);
+ assert sub.fp > fp;
+ scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0));
+ }
+ }
+
+ final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+ final Builder indexBuilder = new Builder(FST.INPUT_TYPE.BYTE1,
+ 0, 0, true, false, Integer.MAX_VALUE,
+ 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), new BytesRef(bytes, 0, bytes.length));
+ 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 extends TermsConsumer {
+ private final FieldInfo fieldInfo;
+ private long numTerms;
+ long sumTotalTermFreq;
+ long sumDocFreq;
+ int docCount;
+ 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
The .tim file contains the list of terms in each
+ * field along with per-term statistics (such as docfreq)
+ * and pointers to the frequencies, positions, payload and
+ * skip data in the .doc, .pos, and .pay files.
+ * See {@link TempBlockTermsWriter} for more details on the format.
+ *
+ *
+ *
NOTE: The term dictionary can plug into different postings implementations:
+ * the postings writer/reader are actually responsible for encoding
+ * and decoding the Postings Metadata and Term Metadata sections described here:
+ *
+ *
+ *
Postings Metadata --> Header, PackedBlockSize
+ *
Term Metadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?,
+ * SkipFPDelta?
Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information
+ * for the postings.
+ *
PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width is
+ * determined by the largest integer. Smaller block size result in smaller variance among width
+ * of integers hence smaller indexes. Larger block size result in more efficient bulk i/o hence
+ * better acceleration. This value should always be a multiple of 64, currently fixed as 128 as
+ * a tradeoff. It is also the skip interval used to accelerate {@link DocsEnum#advance(int)}.
+ *
DocFPDelta determines the position of this term's TermFreqs within the .doc file.
+ * In particular, it is the difference of file offset between this term's
+ * data and previous term's data (or zero, for the first term in the block).On disk it is
+ * stored as the difference from previous value in sequence.
+ *
PosFPDelta determines the position of this term's TermPositions within the .pos file.
+ * While PayFPDelta determines the position of this term's <TermPayloads, TermOffsets?> within
+ * the .pay file. Similar to DocFPDelta, it is the difference between two file positions (or
+ * neglected, for fields that omit payloads and offsets).
+ *
PosVIntBlockFPDelta determines the position of this term's last TermPosition in last pos packed
+ * block within the .pos file. It is synonym for PayVIntBlockFPDelta or OffsetVIntBlockFPDelta.
+ * This is actually used to indicate whether it is necessary to load following
+ * payloads and offsets from .pos instead of .pay. Every time a new block of positions are to be
+ * loaded, the PostingsReader will use this value to check whether current block is packed format
+ * or VInt. When packed format, payloads and offsets are fetched from .pay, otherwise from .pos.
+ * (this value is neglected when total number of positions i.e. totalTermFreq is less or equal
+ * to PackedBlockSize).
+ *
SkipFPDelta determines the position of this term's SkipData within the .doc
+ * file. In particular, it is the length of the TermFreq data.
+ * SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
+ * (i.e. 8 in TempPostingsFormat).
+ *
SingletonDocID is an optimization when a term only appears in one document. In this case, instead
+ * of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
+ * single document ID is written to the term dictionary.
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * Term Index
+ *
The .tip file contains an index into the term dictionary, so that it can be
+ * accessed randomly. See {@link TempBlockTermsWriter} for more details on the format.
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * Frequencies and Skip Data
+ *
+ *
The .doc file contains the lists of documents which contain each term, along
+ * with the frequency of the term in that document (except when frequencies are
+ * omitted: {@link IndexOptions#DOCS_ONLY}). It also saves skip data to the beginning of
+ * each packed or VInt block, when the length of document list is larger than packed block size.
PackedDocDeltaBlock is theoretically generated from two steps:
+ *
+ *
Calculate the difference between each document number and previous one,
+ * and get a d-gaps list (for the first document, use absolute value);
+ *
For those d-gaps from first one to PackedDocBlockNum*PackedBlockSizeth,
+ * separately encode as packed blocks.
+ *
+ * If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step.
+ *
+ *
VIntBlock stores remaining d-gaps (along with frequencies when possible) with a format
+ * that encodes DocDelta and Freq:
+ *
DocDelta: if frequencies are indexed, this determines both the document
+ * number and the frequency. In particular, DocDelta/2 is the difference between
+ * this document number and the previous document number (or zero when this is the
+ * first document in a TermFreqs). When DocDelta is odd, the frequency is one.
+ * When DocDelta is even, the frequency is read as another VInt. If frequencies
+ * are omitted, DocDelta contains the gap (not multiplied by 2) between document
+ * numbers and no frequency information is stored.
+ *
For example, the TermFreqs for a term which occurs once in document seven
+ * and three times in document eleven, with frequencies indexed, would be the
+ * following sequence of VInts:
+ *
15, 8, 3
+ *
If frequencies were omitted ({@link IndexOptions#DOCS_ONLY}) it would be this
+ * sequence of VInts instead:
+ *
7,4
+ *
+ *
PackedDocBlockNum is the number of packed blocks for current term's docids or frequencies.
+ * In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize)
+ *
TrimmedDocFreq = DocFreq % PackedBlockSize == 0 ? DocFreq - 1 : DocFreq.
+ * We use this trick since the definition of skip entry is a little different from base interface.
+ * In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
+ * skipIntervalth, 2*skipIntervalth ... posting in the list. However,
+ * in TempPostingsFormat, the skip data is saved for skipInterval+1th,
+ * 2*skipInterval+1th ... posting (skipInterval==PackedBlockSize in this case).
+ * When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
+ * more skip data than TempSkipWriter.
+ *
SkipDatum is the metadata of one skip entry.
+ * For the first block (no matter packed or VInt), it is omitted.
+ *
DocSkip records the document number of every PackedBlockSizeth document number in
+ * the postings (i.e. last document number in each packed block). On disk it is stored as the
+ * difference from previous value in the sequence.
+ *
DocFPSkip records the file offsets of each block (excluding )posting at
+ * PackedBlockSize+1th, 2*PackedBlockSize+1th ... , in DocFile.
+ * The file offsets are relative to the start of current term's TermFreqs.
+ * On disk it is also stored as the difference from previous SkipDatum in the sequence.
+ *
Since positions and payloads are also block encoded, the skip should skip to related block first,
+ * then fetch the values according to in-block offset. PosFPSkip and PayFPSkip record the file
+ * offsets of related block in .pos and .pay, respectively. While PosBlockOffset indicates
+ * which value to fetch inside the related block (PayBlockOffset is unnecessary since it is always
+ * equal to PosBlockOffset). Same as DocFPSkip, the file offsets are relative to the start of
+ * current term's TermFreqs, and stored as a difference sequence.
+ *
PayByteUpto indicates the start offset of the current payload. It is equivalent to
+ * the sum of the payload lengths in the current block up to PosBlockOffset
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * Positions
+ *
The .pos file contains the lists of positions that each term occurs at within documents. It also
+ * sometimes stores part of payloads and offsets for speedup.
TermPositions are order by term (terms are implicit, from the term dictionary), and position
+ * values for each term document pair are incremental, and ordered by document number.
+ *
PackedPosBlockNum is the number of packed blocks for current term's positions, payloads or offsets.
+ * In particular, PackedPosBlockNum = floor(totalTermFreq/PackedBlockSize)
+ *
PosVIntCount is the number of positions encoded as VInt format. In particular,
+ * PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize
+ *
The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock
+ * in chapter Frequencies and Skip Data.
+ *
PositionDelta is, if payloads are disabled for the term's field, the
+ * difference between the position of the current occurrence in the document and
+ * the previous occurrence (or zero, if this is the first occurrence in this
+ * document). If payloads are enabled for the term's field, then PositionDelta/2
+ * is the difference between the current and the previous position. If payloads
+ * are enabled and PositionDelta is odd, then PayloadLength is stored, indicating
+ * the length of the payload at the current term position.
+ *
For example, the TermPositions for a term which occurs as the fourth term in
+ * one document, and as the fifth and ninth term in a subsequent document, would
+ * be the following sequence of VInts (payloads disabled):
+ *
4, 5, 4
+ *
PayloadData is metadata associated with the current term position. If
+ * PayloadLength is stored at the current position, then it indicates the length
+ * of this payload. If PayloadLength is not stored, then this payload has the same
+ * length as the payload at the previous position.
+ *
OffsetDelta/2 is the difference between this position's startOffset from the
+ * previous occurrence (or zero, if this is the first occurrence in this document).
+ * If OffsetDelta is odd, then the length (endOffset-startOffset) differs from the
+ * previous occurrence and an OffsetLength follows. Offset data is only written for
+ * {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.
+ *
+ *
+ *
+ *
+ *
+ *
+ *
+ * Payloads and Offsets
+ *
The .pay file will store payloads and offsets associated with certain term-document positions.
+ * Some payloads and offsets will be separated out into .pos file, for performance reasons.
The order of TermPayloads/TermOffsets will be the same as TermPositions, note that part of
+ * payload/offsets are stored in .pos.
+ *
The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is the
+ * same as PackedFreqBlock in chapter Frequencies and Skip Data.
+ * While PackedStartDeltaBlock follows a same procedure as PackedDocDeltaBlock.
+ *
PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also synonym
+ * for PackedOffsetBlockNum.
+ *
SumPayLength is the total length of payloads written within one block, should be the sum
+ * of PayLengths in one packed block.
+ *
PayLength in PackedPayLengthBlock is the length of each payload associated with the current
+ * position.
+ *
+ *
+ *
+ *
+ *
+ * @lucene.experimental
+ */
+
+public final class TempPostingsFormat extends PostingsFormat {
+ /**
+ * Filename extension for document number, frequencies, and skip data.
+ * See chapter: Frequencies and Skip Data
+ */
+ public static final String DOC_EXTENSION = "doc";
+
+ /**
+ * Filename extension for positions.
+ * See chapter: Positions
+ */
+ public static final String POS_EXTENSION = "pos";
+
+ /**
+ * Filename extension for payloads and offsets.
+ * See chapter: Payloads and Offsets
+ */
+ public static final String PAY_EXTENSION = "pay";
+
+ private final int minTermBlockSize;
+ private final int maxTermBlockSize;
+
+ /**
+ * Fixed packed block size, number of integers encoded in
+ * a single packed block.
+ */
+ // NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
+ public final static int BLOCK_SIZE = 128;
+
+ /** Creates {@code TempPostingsFormat} with default
+ * settings. */
+ public TempPostingsFormat() {
+ this(TempBlockTermsWriter.DEFAULT_MIN_BLOCK_SIZE, TempBlockTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+ }
+
+ /** Creates {@code TempPostingsFormat} with custom
+ * values for {@code minBlockSize} and {@code
+ * maxBlockSize} passed to block terms dictionary.
+ * @see TempBlockTermsWriter#TempBlockTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
+ public TempPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+ super("TempBlock");
+ this.minTermBlockSize = minTermBlockSize;
+ assert minTermBlockSize > 1;
+ this.maxTermBlockSize = maxTermBlockSize;
+ assert minTermBlockSize <= maxTermBlockSize;
+ }
+
+ @Override
+ public String toString() {
+ return getName() + "(blocksize=" + BLOCK_SIZE + ")";
+ }
+
+ @Override
+ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ TempPostingsWriterBase postingsWriter = new TempPostingsWriter(state);
+
+ boolean success = false;
+ try {
+ FieldsConsumer ret = new TempBlockTermsWriter(state,
+ postingsWriter,
+ minTermBlockSize,
+ maxTermBlockSize);
+ success = true;
+ return ret;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(postingsWriter);
+ }
+ }
+ }
+
+ @Override
+ public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+ TempPostingsReaderBase postingsReader = new TempPostingsReader(state.directory,
+ state.fieldInfos,
+ state.segmentInfo,
+ state.context,
+ state.segmentSuffix);
+ boolean success = false;
+ try {
+ FieldsProducer ret = new TempBlockTermsReader(state.directory,
+ state.fieldInfos,
+ state.segmentInfo,
+ postingsReader,
+ state.context,
+ state.segmentSuffix,
+ state.termsIndexDivisor);
+ success = true;
+ return ret;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(postingsReader);
+ }
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsReader.java
new file mode 100644
index 00000000000..25a0978b36e
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsReader.java
@@ -0,0 +1,1609 @@
+package org.apache.lucene.codecs.temp;
+
+/*
+ * 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 static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.TempPostingsReaderBase;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+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.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+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.codecs.lucene41.ForUtil;
+import org.apache.lucene.codecs.lucene41.Lucene41SkipReader;
+
+
+/**
+ * Concrete class that reads docId(maybe frq,pos,offset,payloads) list
+ * with postings format.
+ *
+ * @see Lucene41SkipReader for details
+ * @lucene.experimental
+ */
+public final class TempPostingsReader extends TempPostingsReaderBase {
+
+ private final IndexInput docIn;
+ private final IndexInput posIn;
+ private final IndexInput payIn;
+
+ private final ForUtil forUtil;
+
+ // public static boolean DEBUG = false;
+
+ /** Sole constructor. */
+ public TempPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
+ boolean success = false;
+ IndexInput docIn = null;
+ IndexInput posIn = null;
+ IndexInput payIn = null;
+ try {
+ docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempPostingsFormat.DOC_EXTENSION),
+ ioContext);
+ CodecUtil.checkHeader(docIn,
+ TempPostingsWriter.DOC_CODEC,
+ TempPostingsWriter.VERSION_CURRENT,
+ TempPostingsWriter.VERSION_CURRENT);
+ forUtil = new ForUtil(docIn);
+
+ if (fieldInfos.hasProx()) {
+ posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempPostingsFormat.POS_EXTENSION),
+ ioContext);
+ CodecUtil.checkHeader(posIn,
+ TempPostingsWriter.POS_CODEC,
+ TempPostingsWriter.VERSION_CURRENT,
+ TempPostingsWriter.VERSION_CURRENT);
+
+ if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
+ payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempPostingsFormat.PAY_EXTENSION),
+ ioContext);
+ CodecUtil.checkHeader(payIn,
+ TempPostingsWriter.PAY_CODEC,
+ TempPostingsWriter.VERSION_CURRENT,
+ TempPostingsWriter.VERSION_CURRENT);
+ }
+ }
+
+ this.docIn = docIn;
+ this.posIn = posIn;
+ this.payIn = payIn;
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(docIn, posIn, payIn);
+ }
+ }
+ }
+
+ @Override
+ public void init(IndexInput termsIn) throws IOException {
+ // Make sure we are talking to the matching postings writer
+ CodecUtil.checkHeader(termsIn,
+ TempPostingsWriter.TERMS_CODEC,
+ TempPostingsWriter.VERSION_CURRENT,
+ TempPostingsWriter.VERSION_CURRENT);
+ final int indexBlockSize = termsIn.readVInt();
+ if (indexBlockSize != BLOCK_SIZE) {
+ throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");
+ }
+ }
+
+ /**
+ * Read values that have been written using variable-length encoding instead of bit-packing.
+ */
+ static void readVIntBlock(IndexInput docIn, int[] docBuffer,
+ int[] freqBuffer, int num, boolean indexHasFreq) throws IOException {
+ if (indexHasFreq) {
+ for(int i=0;i>> 1;
+ if ((code & 1) != 0) {
+ freqBuffer[i] = 1;
+ } else {
+ freqBuffer[i] = docIn.readVInt();
+ }
+ }
+ } else {
+ for(int i=0;i docUpto) {
+ // Skipper moved
+ // if (DEBUG) {
+ // System.out.println("skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer());
+ // }
+ assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto;
+ docUpto = newDocUpto;
+
+ // Force to read next block
+ docBufferUpto = BLOCK_SIZE;
+ accum = skipper.getDoc(); // actually, this is just lastSkipEntry
+ docIn.seek(skipper.getDocPointer()); // now point to the block we want to search
+ }
+ // next time we call advance, this is used to
+ // foresee whether skipper is necessary.
+ nextSkipDoc = skipper.getNextSkipDoc();
+ }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ if (docBufferUpto == BLOCK_SIZE) {
+ refillDocs();
+ }
+
+ // Now scan... this is an inlined/pared down version
+ // of nextDoc():
+ while (true) {
+ // if (DEBUG) {
+ // System.out.println(" scan doc=" + accum + " docBufferUpto=" + docBufferUpto);
+ // }
+ accum += docDeltaBuffer[docBufferUpto];
+ docUpto++;
+
+ if (accum >= target) {
+ break;
+ }
+ docBufferUpto++;
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ }
+
+ if (liveDocs == null || liveDocs.get(accum)) {
+ // if (DEBUG) {
+ // System.out.println(" return doc=" + accum);
+ // }
+ freq = freqBuffer[docBufferUpto];
+ docBufferUpto++;
+ return doc = accum;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" now do nextDoc()");
+ // }
+ docBufferUpto++;
+ return nextDoc();
+ }
+ }
+
+ @Override
+ public long cost() {
+ return docFreq;
+ }
+ }
+
+
+ final class BlockDocsAndPositionsEnum extends DocsAndPositionsEnum {
+
+ private final byte[] encoded;
+
+ private final int[] docDeltaBuffer = new int[MAX_DATA_SIZE];
+ private final int[] freqBuffer = new int[MAX_DATA_SIZE];
+ private final int[] posDeltaBuffer = new int[MAX_DATA_SIZE];
+
+ private int docBufferUpto;
+ private int posBufferUpto;
+
+ private Lucene41SkipReader skipper;
+ private boolean skipped;
+
+ final IndexInput startDocIn;
+
+ IndexInput docIn;
+ final IndexInput posIn;
+
+ final boolean indexHasOffsets;
+ final boolean indexHasPayloads;
+
+ private int docFreq; // number of docs in this posting list
+ private long totalTermFreq; // number of positions in this posting list
+ private int docUpto; // how many docs we've read
+ private int doc; // doc we last read
+ private int accum; // accumulator for doc deltas
+ private int freq; // freq we last read
+ private int position; // current position
+
+ // how many positions "behind" we are; nextPosition must
+ // skip these to "catch up":
+ private int posPendingCount;
+
+ // Lazy pos seek: if != -1 then we must seek to this FP
+ // before reading positions:
+ private long posPendingFP;
+
+ // Where this term's postings start in the .doc file:
+ private long docTermStartFP;
+
+ // Where this term's postings start in the .pos file:
+ private long posTermStartFP;
+
+ // Where this term's payloads/offsets start in the .pay
+ // file:
+ private long payTermStartFP;
+
+ // File pointer where the last (vInt encoded) pos delta
+ // block is. We need this to know whether to bulk
+ // decode vs vInt decode the block:
+ private long lastPosBlockFP;
+
+ // Where this term's skip data starts (after
+ // docTermStartFP) in the .doc file (or -1 if there is
+ // no skip data for this term):
+ private long skipOffset;
+
+ private int nextSkipDoc;
+
+ private Bits liveDocs;
+ private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1
+
+ public BlockDocsAndPositionsEnum(FieldInfo fieldInfo) throws IOException {
+ this.startDocIn = TempPostingsReader.this.docIn;
+ this.docIn = null;
+ this.posIn = TempPostingsReader.this.posIn.clone();
+ encoded = new byte[MAX_ENCODED_SIZE];
+ indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+ indexHasPayloads = fieldInfo.hasPayloads();
+ }
+
+ public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
+ return docIn == startDocIn &&
+ indexHasOffsets == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) &&
+ indexHasPayloads == fieldInfo.hasPayloads();
+ }
+
+ public DocsAndPositionsEnum reset(Bits liveDocs, IntBlockTermState termState) throws IOException {
+ this.liveDocs = liveDocs;
+ // if (DEBUG) {
+ // System.out.println(" FPR.reset: termState=" + termState);
+ // }
+ docFreq = termState.docFreq;
+ docTermStartFP = termState.docStartFP;
+ posTermStartFP = termState.posStartFP;
+ payTermStartFP = termState.payStartFP;
+ skipOffset = termState.skipOffset;
+ totalTermFreq = termState.totalTermFreq;
+ singletonDocID = termState.singletonDocID;
+ if (docFreq > 1) {
+ if (docIn == null) {
+ // lazy init
+ docIn = startDocIn.clone();
+ }
+ docIn.seek(docTermStartFP);
+ }
+ posPendingFP = posTermStartFP;
+ posPendingCount = 0;
+ if (termState.totalTermFreq < BLOCK_SIZE) {
+ lastPosBlockFP = posTermStartFP;
+ } else if (termState.totalTermFreq == BLOCK_SIZE) {
+ lastPosBlockFP = -1;
+ } else {
+ lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
+ }
+
+ doc = -1;
+ accum = 0;
+ docUpto = 0;
+ nextSkipDoc = BLOCK_SIZE - 1;
+ docBufferUpto = BLOCK_SIZE;
+ skipped = false;
+ return this;
+ }
+
+ @Override
+ public int freq() throws IOException {
+ return freq;
+ }
+
+ @Override
+ public int docID() {
+ return doc;
+ }
+
+ private void refillDocs() throws IOException {
+ final int left = docFreq - docUpto;
+ assert left > 0;
+
+ if (left >= BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" fill doc block from fp=" + docIn.getFilePointer());
+ // }
+ forUtil.readBlock(docIn, encoded, docDeltaBuffer);
+ // if (DEBUG) {
+ // System.out.println(" fill freq block from fp=" + docIn.getFilePointer());
+ // }
+ forUtil.readBlock(docIn, encoded, freqBuffer);
+ } else if (docFreq == 1) {
+ docDeltaBuffer[0] = singletonDocID;
+ freqBuffer[0] = (int) totalTermFreq;
+ } else {
+ // Read vInts:
+ // if (DEBUG) {
+ // System.out.println(" fill last vInt doc block from fp=" + docIn.getFilePointer());
+ // }
+ readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
+ }
+ docBufferUpto = 0;
+ }
+
+ private void refillPositions() throws IOException {
+ // if (DEBUG) {
+ // System.out.println(" refillPositions");
+ // }
+ if (posIn.getFilePointer() == lastPosBlockFP) {
+ // if (DEBUG) {
+ // System.out.println(" vInt pos block @ fp=" + posIn.getFilePointer() + " hasPayloads=" + indexHasPayloads + " hasOffsets=" + indexHasOffsets);
+ // }
+ final int count = (int) (totalTermFreq % BLOCK_SIZE);
+ int payloadLength = 0;
+ for(int i=0;i>> 1;
+ if (payloadLength != 0) {
+ posIn.seek(posIn.getFilePointer() + payloadLength);
+ }
+ } else {
+ posDeltaBuffer[i] = code;
+ }
+ if (indexHasOffsets) {
+ if ((posIn.readVInt() & 1) != 0) {
+ // offset length changed
+ posIn.readVInt();
+ }
+ }
+ }
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" bulk pos block @ fp=" + posIn.getFilePointer());
+ // }
+ forUtil.readBlock(posIn, encoded, posDeltaBuffer);
+ }
+ }
+
+ @Override
+ public int nextDoc() throws IOException {
+ // if (DEBUG) {
+ // System.out.println(" FPR.nextDoc");
+ // }
+ while (true) {
+ // if (DEBUG) {
+ // System.out.println(" docUpto=" + docUpto + " (of df=" + docFreq + ") docBufferUpto=" + docBufferUpto);
+ // }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ if (docBufferUpto == BLOCK_SIZE) {
+ refillDocs();
+ }
+ // if (DEBUG) {
+ // System.out.println(" accum=" + accum + " docDeltaBuffer[" + docBufferUpto + "]=" + docDeltaBuffer[docBufferUpto]);
+ // }
+ accum += docDeltaBuffer[docBufferUpto];
+ freq = freqBuffer[docBufferUpto];
+ posPendingCount += freq;
+ docBufferUpto++;
+ docUpto++;
+
+ if (liveDocs == null || liveDocs.get(accum)) {
+ doc = accum;
+ position = 0;
+ // if (DEBUG) {
+ // System.out.println(" return doc=" + doc + " freq=" + freq + " posPendingCount=" + posPendingCount);
+ // }
+ return doc;
+ }
+ // if (DEBUG) {
+ // System.out.println(" doc=" + accum + " is deleted; try next doc");
+ // }
+ }
+ }
+
+ @Override
+ public int advance(int target) throws IOException {
+ // TODO: make frq block load lazy/skippable
+ // if (DEBUG) {
+ // System.out.println(" FPR.advance target=" + target);
+ // }
+
+ if (docFreq > BLOCK_SIZE && target > nextSkipDoc) {
+ // if (DEBUG) {
+ // System.out.println(" try skipper");
+ // }
+ if (skipper == null) {
+ // Lazy init: first time this enum has ever been used for skipping
+ // if (DEBUG) {
+ // System.out.println(" create skipper");
+ // }
+ skipper = new Lucene41SkipReader(docIn.clone(),
+ TempPostingsWriter.maxSkipLevels,
+ BLOCK_SIZE,
+ true,
+ indexHasOffsets,
+ indexHasPayloads);
+ }
+
+ if (!skipped) {
+ assert skipOffset != -1;
+ // This is the first time this enum has skipped
+ // since reset() was called; load the skip data:
+ // if (DEBUG) {
+ // System.out.println(" init skipper");
+ // }
+ skipper.init(docTermStartFP+skipOffset, docTermStartFP, posTermStartFP, payTermStartFP, docFreq);
+ skipped = true;
+ }
+
+ final int newDocUpto = skipper.skipTo(target) + 1;
+
+ if (newDocUpto > docUpto) {
+ // Skipper moved
+ // if (DEBUG) {
+ // System.out.println(" skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer() + " pos.fp=" + skipper.getPosPointer() + " pos.bufferUpto=" + skipper.getPosBufferUpto());
+ // }
+
+ assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto;
+ docUpto = newDocUpto;
+
+ // Force to read next block
+ docBufferUpto = BLOCK_SIZE;
+ accum = skipper.getDoc();
+ docIn.seek(skipper.getDocPointer());
+ posPendingFP = skipper.getPosPointer();
+ posPendingCount = skipper.getPosBufferUpto();
+ }
+ nextSkipDoc = skipper.getNextSkipDoc();
+ }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ if (docBufferUpto == BLOCK_SIZE) {
+ refillDocs();
+ }
+
+ // Now scan... this is an inlined/pared down version
+ // of nextDoc():
+ while (true) {
+ // if (DEBUG) {
+ // System.out.println(" scan doc=" + accum + " docBufferUpto=" + docBufferUpto);
+ // }
+ accum += docDeltaBuffer[docBufferUpto];
+ freq = freqBuffer[docBufferUpto];
+ posPendingCount += freq;
+ docBufferUpto++;
+ docUpto++;
+
+ if (accum >= target) {
+ break;
+ }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ }
+
+ if (liveDocs == null || liveDocs.get(accum)) {
+ // if (DEBUG) {
+ // System.out.println(" return doc=" + accum);
+ // }
+ position = 0;
+ return doc = accum;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" now do nextDoc()");
+ // }
+ return nextDoc();
+ }
+ }
+
+ // TODO: in theory we could avoid loading frq block
+ // when not needed, ie, use skip data to load how far to
+ // seek the pos pointer ... instead of having to load frq
+ // blocks only to sum up how many positions to skip
+ private void skipPositions() throws IOException {
+ // Skip positions now:
+ int toSkip = posPendingCount - freq;
+ // if (DEBUG) {
+ // System.out.println(" FPR.skipPositions: toSkip=" + toSkip);
+ // }
+
+ final int leftInBlock = BLOCK_SIZE - posBufferUpto;
+ if (toSkip < leftInBlock) {
+ posBufferUpto += toSkip;
+ // if (DEBUG) {
+ // System.out.println(" skip w/in block to posBufferUpto=" + posBufferUpto);
+ // }
+ } else {
+ toSkip -= leftInBlock;
+ while(toSkip >= BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" skip whole block @ fp=" + posIn.getFilePointer());
+ // }
+ assert posIn.getFilePointer() != lastPosBlockFP;
+ forUtil.skipBlock(posIn);
+ toSkip -= BLOCK_SIZE;
+ }
+ refillPositions();
+ posBufferUpto = toSkip;
+ // if (DEBUG) {
+ // System.out.println(" skip w/in block to posBufferUpto=" + posBufferUpto);
+ // }
+ }
+
+ position = 0;
+ }
+
+ @Override
+ public int nextPosition() throws IOException {
+ // if (DEBUG) {
+ // System.out.println(" FPR.nextPosition posPendingCount=" + posPendingCount + " posBufferUpto=" + posBufferUpto);
+ // }
+ if (posPendingFP != -1) {
+ // if (DEBUG) {
+ // System.out.println(" seek to pendingFP=" + posPendingFP);
+ // }
+ posIn.seek(posPendingFP);
+ posPendingFP = -1;
+
+ // Force buffer refill:
+ posBufferUpto = BLOCK_SIZE;
+ }
+
+ if (posPendingCount > freq) {
+ skipPositions();
+ posPendingCount = freq;
+ }
+
+ if (posBufferUpto == BLOCK_SIZE) {
+ refillPositions();
+ posBufferUpto = 0;
+ }
+ position += posDeltaBuffer[posBufferUpto++];
+ posPendingCount--;
+ // if (DEBUG) {
+ // System.out.println(" return pos=" + position);
+ // }
+ return position;
+ }
+
+ @Override
+ public int startOffset() {
+ return -1;
+ }
+
+ @Override
+ public int endOffset() {
+ return -1;
+ }
+
+ @Override
+ public BytesRef getPayload() {
+ return null;
+ }
+
+ @Override
+ public long cost() {
+ return docFreq;
+ }
+ }
+
+ // Also handles payloads + offsets
+ final class EverythingEnum extends DocsAndPositionsEnum {
+
+ private final byte[] encoded;
+
+ private final int[] docDeltaBuffer = new int[MAX_DATA_SIZE];
+ private final int[] freqBuffer = new int[MAX_DATA_SIZE];
+ private final int[] posDeltaBuffer = new int[MAX_DATA_SIZE];
+
+ private final int[] payloadLengthBuffer;
+ private final int[] offsetStartDeltaBuffer;
+ private final int[] offsetLengthBuffer;
+
+ private byte[] payloadBytes;
+ private int payloadByteUpto;
+ private int payloadLength;
+
+ private int lastStartOffset;
+ private int startOffset;
+ private int endOffset;
+
+ private int docBufferUpto;
+ private int posBufferUpto;
+
+ private Lucene41SkipReader skipper;
+ private boolean skipped;
+
+ final IndexInput startDocIn;
+
+ IndexInput docIn;
+ final IndexInput posIn;
+ final IndexInput payIn;
+ final BytesRef payload;
+
+ final boolean indexHasOffsets;
+ final boolean indexHasPayloads;
+
+ private int docFreq; // number of docs in this posting list
+ private long totalTermFreq; // number of positions in this posting list
+ private int docUpto; // how many docs we've read
+ private int doc; // doc we last read
+ private int accum; // accumulator for doc deltas
+ private int freq; // freq we last read
+ private int position; // current position
+
+ // how many positions "behind" we are; nextPosition must
+ // skip these to "catch up":
+ private int posPendingCount;
+
+ // Lazy pos seek: if != -1 then we must seek to this FP
+ // before reading positions:
+ private long posPendingFP;
+
+ // Lazy pay seek: if != -1 then we must seek to this FP
+ // before reading payloads/offsets:
+ private long payPendingFP;
+
+ // Where this term's postings start in the .doc file:
+ private long docTermStartFP;
+
+ // Where this term's postings start in the .pos file:
+ private long posTermStartFP;
+
+ // Where this term's payloads/offsets start in the .pay
+ // file:
+ private long payTermStartFP;
+
+ // File pointer where the last (vInt encoded) pos delta
+ // block is. We need this to know whether to bulk
+ // decode vs vInt decode the block:
+ private long lastPosBlockFP;
+
+ // Where this term's skip data starts (after
+ // docTermStartFP) in the .doc file (or -1 if there is
+ // no skip data for this term):
+ private long skipOffset;
+
+ private int nextSkipDoc;
+
+ private Bits liveDocs;
+
+ private boolean needsOffsets; // true if we actually need offsets
+ private boolean needsPayloads; // true if we actually need payloads
+ private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1
+
+ public EverythingEnum(FieldInfo fieldInfo) throws IOException {
+ this.startDocIn = TempPostingsReader.this.docIn;
+ this.docIn = null;
+ this.posIn = TempPostingsReader.this.posIn.clone();
+ this.payIn = TempPostingsReader.this.payIn.clone();
+ encoded = new byte[MAX_ENCODED_SIZE];
+ indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+ if (indexHasOffsets) {
+ offsetStartDeltaBuffer = new int[MAX_DATA_SIZE];
+ offsetLengthBuffer = new int[MAX_DATA_SIZE];
+ } else {
+ offsetStartDeltaBuffer = null;
+ offsetLengthBuffer = null;
+ startOffset = -1;
+ endOffset = -1;
+ }
+
+ indexHasPayloads = fieldInfo.hasPayloads();
+ if (indexHasPayloads) {
+ payloadLengthBuffer = new int[MAX_DATA_SIZE];
+ payloadBytes = new byte[128];
+ payload = new BytesRef();
+ } else {
+ payloadLengthBuffer = null;
+ payloadBytes = null;
+ payload = null;
+ }
+ }
+
+ public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
+ return docIn == startDocIn &&
+ indexHasOffsets == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) &&
+ indexHasPayloads == fieldInfo.hasPayloads();
+ }
+
+ public EverythingEnum reset(Bits liveDocs, IntBlockTermState termState, int flags) throws IOException {
+ this.liveDocs = liveDocs;
+ // if (DEBUG) {
+ // System.out.println(" FPR.reset: termState=" + termState);
+ // }
+ docFreq = termState.docFreq;
+ docTermStartFP = termState.docStartFP;
+ posTermStartFP = termState.posStartFP;
+ payTermStartFP = termState.payStartFP;
+ skipOffset = termState.skipOffset;
+ totalTermFreq = termState.totalTermFreq;
+ singletonDocID = termState.singletonDocID;
+ if (docFreq > 1) {
+ if (docIn == null) {
+ // lazy init
+ docIn = startDocIn.clone();
+ }
+ docIn.seek(docTermStartFP);
+ }
+ posPendingFP = posTermStartFP;
+ payPendingFP = payTermStartFP;
+ posPendingCount = 0;
+ if (termState.totalTermFreq < BLOCK_SIZE) {
+ lastPosBlockFP = posTermStartFP;
+ } else if (termState.totalTermFreq == BLOCK_SIZE) {
+ lastPosBlockFP = -1;
+ } else {
+ lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
+ }
+
+ this.needsOffsets = (flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0;
+ this.needsPayloads = (flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0;
+
+ doc = -1;
+ accum = 0;
+ docUpto = 0;
+ nextSkipDoc = BLOCK_SIZE - 1;
+ docBufferUpto = BLOCK_SIZE;
+ skipped = false;
+ return this;
+ }
+
+ @Override
+ public int freq() throws IOException {
+ return freq;
+ }
+
+ @Override
+ public int docID() {
+ return doc;
+ }
+
+ private void refillDocs() throws IOException {
+ final int left = docFreq - docUpto;
+ assert left > 0;
+
+ if (left >= BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" fill doc block from fp=" + docIn.getFilePointer());
+ // }
+ forUtil.readBlock(docIn, encoded, docDeltaBuffer);
+ // if (DEBUG) {
+ // System.out.println(" fill freq block from fp=" + docIn.getFilePointer());
+ // }
+ forUtil.readBlock(docIn, encoded, freqBuffer);
+ } else if (docFreq == 1) {
+ docDeltaBuffer[0] = singletonDocID;
+ freqBuffer[0] = (int) totalTermFreq;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" fill last vInt doc block from fp=" + docIn.getFilePointer());
+ // }
+ readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
+ }
+ docBufferUpto = 0;
+ }
+
+ private void refillPositions() throws IOException {
+ // if (DEBUG) {
+ // System.out.println(" refillPositions");
+ // }
+ if (posIn.getFilePointer() == lastPosBlockFP) {
+ // if (DEBUG) {
+ // System.out.println(" vInt pos block @ fp=" + posIn.getFilePointer() + " hasPayloads=" + indexHasPayloads + " hasOffsets=" + indexHasOffsets);
+ // }
+ final int count = (int) (totalTermFreq % BLOCK_SIZE);
+ int payloadLength = 0;
+ int offsetLength = 0;
+ payloadByteUpto = 0;
+ for(int i=0;i docUpto) {
+ // Skipper moved
+ // if (DEBUG) {
+ // System.out.println(" skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer() + " pos.fp=" + skipper.getPosPointer() + " pos.bufferUpto=" + skipper.getPosBufferUpto() + " pay.fp=" + skipper.getPayPointer() + " lastStartOffset=" + lastStartOffset);
+ // }
+ assert newDocUpto % BLOCK_SIZE == 0 : "got " + newDocUpto;
+ docUpto = newDocUpto;
+
+ // Force to read next block
+ docBufferUpto = BLOCK_SIZE;
+ accum = skipper.getDoc();
+ docIn.seek(skipper.getDocPointer());
+ posPendingFP = skipper.getPosPointer();
+ payPendingFP = skipper.getPayPointer();
+ posPendingCount = skipper.getPosBufferUpto();
+ lastStartOffset = 0; // new document
+ payloadByteUpto = skipper.getPayloadByteUpto();
+ }
+ nextSkipDoc = skipper.getNextSkipDoc();
+ }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ if (docBufferUpto == BLOCK_SIZE) {
+ refillDocs();
+ }
+
+ // Now scan:
+ while (true) {
+ // if (DEBUG) {
+ // System.out.println(" scan doc=" + accum + " docBufferUpto=" + docBufferUpto);
+ // }
+ accum += docDeltaBuffer[docBufferUpto];
+ freq = freqBuffer[docBufferUpto];
+ posPendingCount += freq;
+ docBufferUpto++;
+ docUpto++;
+
+ if (accum >= target) {
+ break;
+ }
+ if (docUpto == docFreq) {
+ return doc = NO_MORE_DOCS;
+ }
+ }
+
+ if (liveDocs == null || liveDocs.get(accum)) {
+ // if (DEBUG) {
+ // System.out.println(" return doc=" + accum);
+ // }
+ position = 0;
+ lastStartOffset = 0;
+ return doc = accum;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" now do nextDoc()");
+ // }
+ return nextDoc();
+ }
+ }
+
+ // TODO: in theory we could avoid loading frq block
+ // when not needed, ie, use skip data to load how far to
+ // seek the pos pointer ... instead of having to load frq
+ // blocks only to sum up how many positions to skip
+ private void skipPositions() throws IOException {
+ // Skip positions now:
+ int toSkip = posPendingCount - freq;
+ // if (DEBUG) {
+ // System.out.println(" FPR.skipPositions: toSkip=" + toSkip);
+ // }
+
+ final int leftInBlock = BLOCK_SIZE - posBufferUpto;
+ if (toSkip < leftInBlock) {
+ int end = posBufferUpto + toSkip;
+ while(posBufferUpto < end) {
+ if (indexHasPayloads) {
+ payloadByteUpto += payloadLengthBuffer[posBufferUpto];
+ }
+ posBufferUpto++;
+ }
+ // if (DEBUG) {
+ // System.out.println(" skip w/in block to posBufferUpto=" + posBufferUpto);
+ // }
+ } else {
+ toSkip -= leftInBlock;
+ while(toSkip >= BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" skip whole block @ fp=" + posIn.getFilePointer());
+ // }
+ assert posIn.getFilePointer() != lastPosBlockFP;
+ forUtil.skipBlock(posIn);
+
+ if (indexHasPayloads) {
+ // Skip payloadLength block:
+ forUtil.skipBlock(payIn);
+
+ // Skip payloadBytes block:
+ int numBytes = payIn.readVInt();
+ payIn.seek(payIn.getFilePointer() + numBytes);
+ }
+
+ if (indexHasOffsets) {
+ forUtil.skipBlock(payIn);
+ forUtil.skipBlock(payIn);
+ }
+ toSkip -= BLOCK_SIZE;
+ }
+ refillPositions();
+ payloadByteUpto = 0;
+ posBufferUpto = 0;
+ while(posBufferUpto < toSkip) {
+ if (indexHasPayloads) {
+ payloadByteUpto += payloadLengthBuffer[posBufferUpto];
+ }
+ posBufferUpto++;
+ }
+ // if (DEBUG) {
+ // System.out.println(" skip w/in block to posBufferUpto=" + posBufferUpto);
+ // }
+ }
+
+ position = 0;
+ lastStartOffset = 0;
+ }
+
+ @Override
+ public int nextPosition() throws IOException {
+ // if (DEBUG) {
+ // System.out.println(" FPR.nextPosition posPendingCount=" + posPendingCount + " posBufferUpto=" + posBufferUpto + " payloadByteUpto=" + payloadByteUpto)// ;
+ // }
+ if (posPendingFP != -1) {
+ // if (DEBUG) {
+ // System.out.println(" seek pos to pendingFP=" + posPendingFP);
+ // }
+ posIn.seek(posPendingFP);
+ posPendingFP = -1;
+
+ if (payPendingFP != -1) {
+ // if (DEBUG) {
+ // System.out.println(" seek pay to pendingFP=" + payPendingFP);
+ // }
+ payIn.seek(payPendingFP);
+ payPendingFP = -1;
+ }
+
+ // Force buffer refill:
+ posBufferUpto = BLOCK_SIZE;
+ }
+
+ if (posPendingCount > freq) {
+ skipPositions();
+ posPendingCount = freq;
+ }
+
+ if (posBufferUpto == BLOCK_SIZE) {
+ refillPositions();
+ posBufferUpto = 0;
+ }
+ position += posDeltaBuffer[posBufferUpto];
+
+ if (indexHasPayloads) {
+ payloadLength = payloadLengthBuffer[posBufferUpto];
+ payload.bytes = payloadBytes;
+ payload.offset = payloadByteUpto;
+ payload.length = payloadLength;
+ payloadByteUpto += payloadLength;
+ }
+
+ if (indexHasOffsets) {
+ startOffset = lastStartOffset + offsetStartDeltaBuffer[posBufferUpto];
+ endOffset = startOffset + offsetLengthBuffer[posBufferUpto];
+ lastStartOffset = startOffset;
+ }
+
+ posBufferUpto++;
+ posPendingCount--;
+ // if (DEBUG) {
+ // System.out.println(" return pos=" + position);
+ // }
+ return position;
+ }
+
+ @Override
+ public int startOffset() {
+ return startOffset;
+ }
+
+ @Override
+ public int endOffset() {
+ return endOffset;
+ }
+
+ @Override
+ public BytesRef getPayload() {
+ // if (DEBUG) {
+ // System.out.println(" FPR.getPayload payloadLength=" + payloadLength + " payloadByteUpto=" + payloadByteUpto);
+ // }
+ if (payloadLength == 0) {
+ return null;
+ } else {
+ return payload;
+ }
+ }
+
+ @Override
+ public long cost() {
+ return docFreq;
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java
new file mode 100644
index 00000000000..0d0d4bb8a53
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/temp/TempPostingsWriter.java
@@ -0,0 +1,586 @@
+package org.apache.lucene.codecs.temp;
+
+/*
+ * 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 static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.TempPostingsWriterBase;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.lucene41.Lucene41SkipWriter;
+import org.apache.lucene.codecs.lucene41.ForUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+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.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+
+/**
+ * Concrete class that writes docId(maybe frq,pos,offset,payloads) list
+ * with postings format.
+ *
+ * Postings list for each term will be stored separately.
+ *
+ * @see Lucene41SkipWriter for details about skipping setting and postings layout.
+ * @lucene.experimental
+ */
+public final class TempPostingsWriter extends TempPostingsWriterBase {
+
+ /**
+ * Expert: The maximum number of skip levels. Smaller values result in
+ * slightly smaller indexes, but slower skipping in big posting lists.
+ */
+ static final int maxSkipLevels = 10;
+
+ final static String TERMS_CODEC = "TempPostingsWriterTerms";
+ final static String DOC_CODEC = "TempPostingsWriterDoc";
+ final static String POS_CODEC = "TempPostingsWriterPos";
+ final static String PAY_CODEC = "TempPostingsWriterPay";
+
+ // Increment version to change it
+ final static int VERSION_START = 0;
+ final static int VERSION_CURRENT = VERSION_START;
+
+ final IndexOutput docOut;
+ final IndexOutput posOut;
+ final IndexOutput payOut;
+
+ private IndexOutput termsOut;
+
+ // How current field indexes postings:
+ private boolean fieldHasFreqs;
+ private boolean fieldHasPositions;
+ private boolean fieldHasOffsets;
+ private boolean fieldHasPayloads;
+
+ // Holds starting file pointers for each term:
+ private long docTermStartFP;
+ private long posTermStartFP;
+ private long payTermStartFP;
+
+ final int[] docDeltaBuffer;
+ final int[] freqBuffer;
+ private int docBufferUpto;
+
+ final int[] posDeltaBuffer;
+ final int[] payloadLengthBuffer;
+ final int[] offsetStartDeltaBuffer;
+ final int[] offsetLengthBuffer;
+ private int posBufferUpto;
+
+ private byte[] payloadBytes;
+ private int payloadByteUpto;
+
+ private int lastBlockDocID;
+ private long lastBlockPosFP;
+ private long lastBlockPayFP;
+ private int lastBlockPosBufferUpto;
+ private int lastBlockPayloadByteUpto;
+
+ private int lastDocID;
+ private int lastPosition;
+ private int lastStartOffset;
+ private int docCount;
+
+ final byte[] encoded;
+
+ private final ForUtil forUtil;
+ private final Lucene41SkipWriter skipWriter;
+
+ /** Creates a postings writer with the specified PackedInts overhead ratio */
+ // TODO: does this ctor even make sense?
+ public TempPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
+ super();
+
+ docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempPostingsFormat.DOC_EXTENSION),
+ state.context);
+ IndexOutput posOut = null;
+ IndexOutput payOut = null;
+ boolean success = false;
+ try {
+ CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
+ forUtil = new ForUtil(acceptableOverheadRatio, docOut);
+ if (state.fieldInfos.hasProx()) {
+ posDeltaBuffer = new int[MAX_DATA_SIZE];
+ posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempPostingsFormat.POS_EXTENSION),
+ state.context);
+ CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
+
+ if (state.fieldInfos.hasPayloads()) {
+ payloadBytes = new byte[128];
+ payloadLengthBuffer = new int[MAX_DATA_SIZE];
+ } else {
+ payloadBytes = null;
+ payloadLengthBuffer = null;
+ }
+
+ if (state.fieldInfos.hasOffsets()) {
+ offsetStartDeltaBuffer = new int[MAX_DATA_SIZE];
+ offsetLengthBuffer = new int[MAX_DATA_SIZE];
+ } else {
+ offsetStartDeltaBuffer = null;
+ offsetLengthBuffer = null;
+ }
+
+ if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
+ payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempPostingsFormat.PAY_EXTENSION),
+ state.context);
+ CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
+ }
+ } else {
+ posDeltaBuffer = null;
+ payloadLengthBuffer = null;
+ offsetStartDeltaBuffer = null;
+ offsetLengthBuffer = null;
+ payloadBytes = null;
+ }
+ this.payOut = payOut;
+ this.posOut = posOut;
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(docOut, posOut, payOut);
+ }
+ }
+
+ docDeltaBuffer = new int[MAX_DATA_SIZE];
+ freqBuffer = new int[MAX_DATA_SIZE];
+
+ // TODO: should we try skipping every 2/4 blocks...?
+ skipWriter = new Lucene41SkipWriter(maxSkipLevels,
+ BLOCK_SIZE,
+ state.segmentInfo.getDocCount(),
+ docOut,
+ posOut,
+ payOut);
+
+ encoded = new byte[MAX_ENCODED_SIZE];
+ }
+
+ /** Creates a postings writer with PackedInts.COMPACT */
+ public TempPostingsWriter(SegmentWriteState state) throws IOException {
+ this(state, PackedInts.COMPACT);
+ }
+
+ @Override
+ public void start(IndexOutput termsOut) throws IOException {
+ this.termsOut = termsOut;
+ CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
+ termsOut.writeVInt(BLOCK_SIZE);
+ }
+
+ @Override
+ public void setField(FieldInfo fieldInfo) {
+ IndexOptions indexOptions = fieldInfo.getIndexOptions();
+ fieldHasFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+ fieldHasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+ fieldHasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+ fieldHasPayloads = fieldInfo.hasPayloads();
+ skipWriter.setField(fieldHasPositions, fieldHasOffsets, fieldHasPayloads);
+ }
+
+ @Override
+ public void startTerm() {
+ docTermStartFP = docOut.getFilePointer();
+ if (fieldHasPositions) {
+ posTermStartFP = posOut.getFilePointer();
+ if (fieldHasPayloads || fieldHasOffsets) {
+ payTermStartFP = payOut.getFilePointer();
+ }
+ }
+ lastDocID = 0;
+ lastBlockDocID = -1;
+ // if (DEBUG) {
+ // System.out.println("FPW.startTerm startFP=" + docTermStartFP);
+ // }
+ skipWriter.resetSkip();
+ }
+
+ @Override
+ public void startDoc(int docID, int termDocFreq) throws IOException {
+ // if (DEBUG) {
+ // System.out.println("FPW.startDoc docID["+docBufferUpto+"]=" + docID);
+ // }
+ // Have collected a block of docs, and get a new doc.
+ // Should write skip data as well as postings list for
+ // current block.
+ if (lastBlockDocID != -1 && docBufferUpto == 0) {
+ // if (DEBUG) {
+ // System.out.println(" bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-1));
+ // }
+ skipWriter.bufferSkip(lastBlockDocID, docCount, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockPayloadByteUpto);
+ }
+
+ final int docDelta = docID - lastDocID;
+
+ if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
+ throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
+ }
+
+ docDeltaBuffer[docBufferUpto] = docDelta;
+ // if (DEBUG) {
+ // System.out.println(" docDeltaBuffer[" + docBufferUpto + "]=" + docDelta);
+ // }
+ if (fieldHasFreqs) {
+ freqBuffer[docBufferUpto] = termDocFreq;
+ }
+ docBufferUpto++;
+ docCount++;
+
+ if (docBufferUpto == BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" write docDelta block @ fp=" + docOut.getFilePointer());
+ // }
+ forUtil.writeBlock(docDeltaBuffer, encoded, docOut);
+ if (fieldHasFreqs) {
+ // if (DEBUG) {
+ // System.out.println(" write freq block @ fp=" + docOut.getFilePointer());
+ // }
+ forUtil.writeBlock(freqBuffer, encoded, docOut);
+ }
+ // NOTE: don't set docBufferUpto back to 0 here;
+ // finishDoc will do so (because it needs to see that
+ // the block was filled so it can save skip data)
+ }
+
+
+ lastDocID = docID;
+ lastPosition = 0;
+ lastStartOffset = 0;
+ }
+
+ /** Add a new position & payload */
+ @Override
+ public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+ // if (DEBUG) {
+ // System.out.println("FPW.addPosition pos=" + position + " posBufferUpto=" + posBufferUpto + (fieldHasPayloads ? " payloadByteUpto=" + payloadByteUpto: ""));
+ // }
+ posDeltaBuffer[posBufferUpto] = position - lastPosition;
+ if (fieldHasPayloads) {
+ if (payload == null || payload.length == 0) {
+ // no payload
+ payloadLengthBuffer[posBufferUpto] = 0;
+ } else {
+ payloadLengthBuffer[posBufferUpto] = payload.length;
+ if (payloadByteUpto + payload.length > payloadBytes.length) {
+ payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length);
+ }
+ System.arraycopy(payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length);
+ payloadByteUpto += payload.length;
+ }
+ }
+
+ if (fieldHasOffsets) {
+ assert startOffset >= lastStartOffset;
+ assert endOffset >= startOffset;
+ offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset;
+ offsetLengthBuffer[posBufferUpto] = endOffset - startOffset;
+ lastStartOffset = startOffset;
+ }
+
+ posBufferUpto++;
+ lastPosition = position;
+ if (posBufferUpto == BLOCK_SIZE) {
+ // if (DEBUG) {
+ // System.out.println(" write pos bulk block @ fp=" + posOut.getFilePointer());
+ // }
+ forUtil.writeBlock(posDeltaBuffer, encoded, posOut);
+
+ if (fieldHasPayloads) {
+ forUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
+ payOut.writeVInt(payloadByteUpto);
+ payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
+ payloadByteUpto = 0;
+ }
+ if (fieldHasOffsets) {
+ forUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
+ forUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
+ }
+ posBufferUpto = 0;
+ }
+ }
+
+ @Override
+ public void finishDoc() throws IOException {
+ // Since we don't know df for current term, we had to buffer
+ // those skip data for each block, and when a new doc comes,
+ // write them to skip file.
+ if (docBufferUpto == BLOCK_SIZE) {
+ lastBlockDocID = lastDocID;
+ if (posOut != null) {
+ if (payOut != null) {
+ lastBlockPayFP = payOut.getFilePointer();
+ }
+ lastBlockPosFP = posOut.getFilePointer();
+ lastBlockPosBufferUpto = posBufferUpto;
+ lastBlockPayloadByteUpto = payloadByteUpto;
+ }
+ // if (DEBUG) {
+ // System.out.println(" docBufferUpto="+docBufferUpto+" now get lastBlockDocID="+lastBlockDocID+" lastBlockPosFP=" + lastBlockPosFP + " lastBlockPosBufferUpto=" + lastBlockPosBufferUpto + " lastBlockPayloadByteUpto=" + lastBlockPayloadByteUpto);
+ // }
+ docBufferUpto = 0;
+ }
+ }
+
+ private static class PendingTerm {
+ public final long docStartFP;
+ public final long posStartFP;
+ public final long payStartFP;
+ public final long skipOffset;
+ public final long lastPosBlockOffset;
+ public final int singletonDocID;
+
+ public PendingTerm(long docStartFP, long posStartFP, long payStartFP, long skipOffset, long lastPosBlockOffset, int singletonDocID) {
+ this.docStartFP = docStartFP;
+ this.posStartFP = posStartFP;
+ this.payStartFP = payStartFP;
+ this.skipOffset = skipOffset;
+ this.lastPosBlockOffset = lastPosBlockOffset;
+ this.singletonDocID = singletonDocID;
+ }
+ }
+
+ private final List pendingTerms = new ArrayList();
+
+ /** Called when we are done adding docs to this term */
+ @Override
+ public void finishTerm(TermStats stats) throws IOException {
+ assert stats.docFreq > 0;
+
+ // TODO: wasteful we are counting this (counting # docs
+ // for this term) in two places?
+ assert stats.docFreq == docCount: stats.docFreq + " vs " + docCount;
+
+ // if (DEBUG) {
+ // System.out.println("FPW.finishTerm docFreq=" + stats.docFreq);
+ // }
+
+ // if (DEBUG) {
+ // if (docBufferUpto > 0) {
+ // System.out.println(" write doc/freq vInt block (count=" + docBufferUpto + ") at fp=" + docOut.getFilePointer() + " docTermStartFP=" + docTermStartFP);
+ // }
+ // }
+
+ // docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to it.
+ final int singletonDocID;
+ if (stats.docFreq == 1) {
+ // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
+ singletonDocID = docDeltaBuffer[0];
+ } else {
+ singletonDocID = -1;
+ // vInt encode the remaining doc deltas and freqs:
+ for(int i=0;i 0) {
+ // System.out.println(" write pos vInt block (count=" + posBufferUpto + ") at fp=" + posOut.getFilePointer() + " posTermStartFP=" + posTermStartFP + " hasPayloads=" + fieldHasPayloads + " hasOffsets=" + fieldHasOffsets);
+ // }
+ // }
+
+ // totalTermFreq is just total number of positions(or payloads, or offsets)
+ // associated with current term.
+ assert stats.totalTermFreq != -1;
+ if (stats.totalTermFreq > BLOCK_SIZE) {
+ // record file offset for last pos in last block
+ lastPosBlockOffset = posOut.getFilePointer() - posTermStartFP;
+ } else {
+ lastPosBlockOffset = -1;
+ }
+ if (posBufferUpto > 0) {
+ // TODO: should we send offsets/payloads to
+ // .pay...? seems wasteful (have to store extra
+ // vLong for low (< BLOCK_SIZE) DF terms = vast vast
+ // majority)
+
+ // vInt encode the remaining positions/payloads/offsets:
+ int lastPayloadLength = -1; // force first payload length to be written
+ int lastOffsetLength = -1; // force first offset length to be written
+ int payloadBytesReadUpto = 0;
+ for(int i=0;i