diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexOutput.java b/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexOutput.java index ec89356f788..21fc2b80b42 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexOutput.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexOutput.java @@ -24,6 +24,7 @@ package org.apache.lucene.codecs.intblock; import java.io.IOException; import org.apache.lucene.codecs.sep.IntIndexOutput; +import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; /** Abstract base class that writes fixed-size blocks of ints @@ -51,7 +52,7 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput { protected abstract void flushBlock() throws IOException; @Override - public IntIndexOutput.Index index() throws IOException { + public IntIndexOutput.Index index() { return new Index(); } @@ -79,7 +80,7 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput { } @Override - public void write(IndexOutput indexOut, boolean absolute) throws IOException { + public void write(DataOutput indexOut, boolean absolute) throws IOException { if (absolute) { indexOut.writeVInt(upto); indexOut.writeVLong(fp); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexOutput.java b/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexOutput.java index c035e231685..f8d401dc8b8 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexOutput.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexOutput.java @@ -24,6 +24,7 @@ package org.apache.lucene.codecs.intblock; import java.io.IOException; import org.apache.lucene.codecs.sep.IntIndexOutput; +import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; // TODO: much of this can be shared code w/ the fixed case @@ -60,7 +61,7 @@ public abstract class VariableIntBlockIndexOutput extends IntIndexOutput { protected abstract int add(int value) throws IOException; @Override - public IntIndexOutput.Index index() throws IOException { + public IntIndexOutput.Index index() { return new Index(); } @@ -88,7 +89,7 @@ public abstract class VariableIntBlockIndexOutput extends IntIndexOutput { } @Override - public void write(IndexOutput indexOut, boolean absolute) throws IOException { + public void write(DataOutput indexOut, boolean absolute) throws IOException { assert upto >= 0; if (absolute) { indexOut.writeVInt(upto); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/sep/IntIndexOutput.java b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/IntIndexOutput.java index c3f9fe2945f..f6523ee84a0 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/sep/IntIndexOutput.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/IntIndexOutput.java @@ -20,7 +20,7 @@ package org.apache.lucene.codecs.sep; // TODO: we may want tighter integration w/ IndexOutput -- // may give better perf: -import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.DataOutput; import java.io.IOException; import java.io.Closeable; @@ -49,12 +49,12 @@ public abstract class IntIndexOutput implements Closeable { /** Writes "location" of current output pointer of primary * output to different output (out) */ - public abstract void write(IndexOutput indexOut, boolean absolute) throws IOException; + public abstract void write(DataOutput indexOut, boolean absolute) throws IOException; } /** If you are indexing the primary output file, call * this and interact with the returned IndexWriter. */ - public abstract Index index() throws IOException; + public abstract Index index(); @Override public abstract void close() throws IOException; diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java index ef9630244fe..eed2380a6d8 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java @@ -87,6 +87,8 @@ public final class SepPostingsWriter extends PostingsWriterBase { final int totalNumDocs; + PendingTerm lastState; + boolean storePayloads; IndexOptions indexOptions; @@ -116,9 +118,9 @@ public final class SepPostingsWriter extends PostingsWriterBase { this.skipInterval = skipInterval; this.skipMinimum = skipInterval; /* set to the same for now */ final String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, DOC_EXTENSION); + docOut = factory.createOutput(state.directory, docFileName, state.context); docIndex = docOut.index(); - if (state.fieldInfos.hasFreq()) { final String frqFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FREQ_EXTENSION); freqOut = factory.createOutput(state.directory, frqFileName, state.context); @@ -134,7 +136,7 @@ public final class SepPostingsWriter extends PostingsWriterBase { final String payloadFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, PAYLOAD_EXTENSION); payloadOut = state.directory.createOutput(payloadFileName, state.context); } - + final String skipFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SKIP_EXTENSION); skipOut = state.directory.createOutput(skipFileName, state.context); @@ -264,8 +266,8 @@ public final class SepPostingsWriter extends PostingsWriterBase { public final IntIndexOutput.Index docIndex; public final IntIndexOutput.Index freqIndex; public final IntIndexOutput.Index posIndex; - public final long payloadFP; - public final long skipFP; + public long payloadFP; + public long skipFP; public PendingTerm(IntIndexOutput.Index docIndex, IntIndexOutput.Index freqIndex, IntIndexOutput.Index posIndex, long payloadFP, long skipFP) { this.docIndex = docIndex; @@ -331,57 +333,60 @@ public final class SepPostingsWriter extends PostingsWriterBase { final int absStart = pendingTerms.size() - start; final List slice = pendingTerms.subList(absStart, absStart+count); - long lastPayloadFP = 0; - long lastSkipFP = 0; - if (count == 0) { termsOut.writeByte((byte) 0); return; } - final PendingTerm firstTerm = slice.get(0); - final IntIndexOutput.Index docIndexFlush = firstTerm.docIndex; - final IntIndexOutput.Index freqIndexFlush = firstTerm.freqIndex; - final IntIndexOutput.Index posIndexFlush = firstTerm.posIndex; + long lastSkipFP = 0; + long lastPayloadFP = 0; + + boolean isFirstTerm = true; for(int idx=0;idx= doc && docFreq >= skipMinimum) { + + // There are enough docs in the posting to have + // skip data, and its not too close + + if (skipper == null) { + // This DocsEnum has never done any skipping + skipper = new SepSkipListReader(skipIn.clone(), + freqIn, + docIn, + posIn, + maxSkipLevels, skipInterval); + + } + + if (!skipped) { + // We haven't yet skipped for this posting + skipper.init(skipFP, + docIndex, + freqIndex, + posIndex, + 0, + docFreq, + storePayloads); + skipper.setIndexOptions(indexOptions); + + skipped = true; + } + + final int newCount = skipper.skipTo(target); + + if (newCount > count) { + + // Skipper did move + if (!omitTF) { + skipper.getFreqIndex().seek(freqReader); + } + skipper.getDocIndex().seek(docReader); + count = newCount; + doc = accum = skipper.getDoc(); + } + } + + // Now, linear scan for the rest: + do { + if (nextDoc() == NO_MORE_DOCS) { + return NO_MORE_DOCS; + } + } while (target > doc); + + return doc; + } + + @Override + public long cost() { + return docFreq; + } + } + + class SepDocsAndPositionsEnum extends DocsAndPositionsEnum { + int docFreq; + int doc = -1; + int accum; + int count; + int freq; + long freqStart; + + private boolean storePayloads; + private Bits liveDocs; + private final IntIndexInput.Reader docReader; + private final IntIndexInput.Reader freqReader; + private final IntIndexInput.Reader posReader; + private final IndexInput payloadIn; + private long skipFP; + + private final IntIndexInput.Index docIndex; + private final IntIndexInput.Index freqIndex; + private final IntIndexInput.Index posIndex; + private final IntIndexInput startDocIn; + + private long payloadFP; + + private int pendingPosCount; + private int position; + private int payloadLength; + private long pendingPayloadBytes; + + private boolean skipped; + private SepSkipListReader skipper; + private boolean payloadPending; + private boolean posSeekPending; + + SepDocsAndPositionsEnum() throws IOException { + startDocIn = docIn; + docReader = docIn.reader(); + docIndex = docIn.index(); + freqReader = freqIn.reader(); + freqIndex = freqIn.index(); + posReader = posIn.reader(); + posIndex = posIn.index(); + payloadIn = TempSepPostingsReader.this.payloadIn.clone(); + } + + SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException { + this.liveDocs = liveDocs; + storePayloads = fieldInfo.hasPayloads(); + //System.out.println("Sep D&P init"); + + // TODO: can't we only do this if consumer + // skipped consuming the previous docs? + docIndex.copyFrom(termState.docIndex); + docIndex.seek(docReader); + //System.out.println(" docIndex=" + docIndex); + + freqIndex.copyFrom(termState.freqIndex); + freqIndex.seek(freqReader); + //System.out.println(" freqIndex=" + freqIndex); + + posIndex.copyFrom(termState.posIndex); + //System.out.println(" posIndex=" + posIndex); + posSeekPending = true; + payloadPending = false; + + payloadFP = termState.payloadFP; + skipFP = termState.skipFP; + //System.out.println(" skipFP=" + skipFP); + + docFreq = termState.docFreq; + count = 0; + doc = -1; + accum = 0; + pendingPosCount = 0; + pendingPayloadBytes = 0; + skipped = false; + + return this; + } + + @Override + public int nextDoc() throws IOException { + + while(true) { + if (count == docFreq) { + return doc = NO_MORE_DOCS; + } + + count++; + + // TODO: maybe we should do the 1-bit trick for encoding + // freq=1 case? + + // Decode next doc + //System.out.println(" sep d&p read doc"); + accum += docReader.next(); + + //System.out.println(" sep d&p read freq"); + freq = freqReader.next(); + + pendingPosCount += freq; + + if (liveDocs == null || liveDocs.get(accum)) { + break; + } + } + + position = 0; + return (doc = accum); + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int advance(int target) throws IOException { + //System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this); + + if ((target - skipInterval) >= doc && docFreq >= skipMinimum) { + + // There are enough docs in the posting to have + // skip data, and its not too close + + if (skipper == null) { + //System.out.println(" create skipper"); + // This DocsEnum has never done any skipping + skipper = new SepSkipListReader(skipIn.clone(), + freqIn, + docIn, + posIn, + maxSkipLevels, skipInterval); + } + + if (!skipped) { + //System.out.println(" init skip data skipFP=" + skipFP); + // We haven't yet skipped for this posting + skipper.init(skipFP, + docIndex, + freqIndex, + posIndex, + payloadFP, + docFreq, + storePayloads); + skipper.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS); + skipped = true; + } + final int newCount = skipper.skipTo(target); + //System.out.println(" skip newCount=" + newCount + " vs " + count); + + if (newCount > count) { + + // Skipper did move + skipper.getFreqIndex().seek(freqReader); + skipper.getDocIndex().seek(docReader); + //System.out.println(" doc seek'd to " + skipper.getDocIndex()); + // NOTE: don't seek pos here; do it lazily + // instead. Eg a PhraseQuery may skip to many + // docs before finally asking for positions... + posIndex.copyFrom(skipper.getPosIndex()); + posSeekPending = true; + count = newCount; + doc = accum = skipper.getDoc(); + //System.out.println(" moved to doc=" + doc); + //payloadIn.seek(skipper.getPayloadPointer()); + payloadFP = skipper.getPayloadPointer(); + pendingPosCount = 0; + pendingPayloadBytes = 0; + payloadPending = false; + payloadLength = skipper.getPayloadLength(); + //System.out.println(" move payloadLen=" + payloadLength); + } + } + + // Now, linear scan for the rest: + do { + if (nextDoc() == NO_MORE_DOCS) { + //System.out.println(" advance nextDoc=END"); + return NO_MORE_DOCS; + } + //System.out.println(" advance nextDoc=" + doc); + } while (target > doc); + + //System.out.println(" return doc=" + doc); + return doc; + } + + @Override + public int nextPosition() throws IOException { + if (posSeekPending) { + posIndex.seek(posReader); + payloadIn.seek(payloadFP); + posSeekPending = false; + } + + // scan over any docs that were iterated without their + // positions + while (pendingPosCount > freq) { + final int code = posReader.next(); + if (storePayloads && (code & 1) != 0) { + // Payload length has changed + payloadLength = posReader.next(); + assert payloadLength >= 0; + } + pendingPosCount--; + position = 0; + pendingPayloadBytes += payloadLength; + } + + final int code = posReader.next(); + + if (storePayloads) { + if ((code & 1) != 0) { + // Payload length has changed + payloadLength = posReader.next(); + assert payloadLength >= 0; + } + position += code >>> 1; + pendingPayloadBytes += payloadLength; + payloadPending = payloadLength > 0; + } else { + position += code; + } + + pendingPosCount--; + assert pendingPosCount >= 0; + return position; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + private BytesRef payload; + + @Override + public BytesRef getPayload() throws IOException { + if (!payloadPending) { + return null; + } + + if (pendingPayloadBytes == 0) { + return payload; + } + + assert pendingPayloadBytes >= payloadLength; + + if (pendingPayloadBytes > payloadLength) { + payloadIn.seek(payloadIn.getFilePointer() + (pendingPayloadBytes - payloadLength)); + } + + if (payload == null) { + payload = new BytesRef(); + payload.bytes = new byte[payloadLength]; + } else if (payload.bytes.length < payloadLength) { + payload.grow(payloadLength); + } + + payloadIn.readBytes(payload.bytes, 0, payloadLength); + payload.length = payloadLength; + pendingPayloadBytes = 0; + return payload; + } + + @Override + public long cost() { + return docFreq; + } + } +} diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/sep/TempSepPostingsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/TempSepPostingsWriter.java new file mode 100644 index 00000000000..f7888331df7 --- /dev/null +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/sep/TempSepPostingsWriter.java @@ -0,0 +1,375 @@ +package org.apache.lucene.codecs.sep; + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.TempPostingsWriterBase; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.sep.*; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocsEnum; +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.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; + +/** Writes frq to .frq, docs to .doc, pos to .pos, payloads + * to .pyl, skip data to .skp + * + * @lucene.experimental */ +public final class TempSepPostingsWriter extends TempPostingsWriterBase { + final static String CODEC = "TempSepPostingsWriter"; + + final static String DOC_EXTENSION = "doc"; + final static String SKIP_EXTENSION = "skp"; + final static String FREQ_EXTENSION = "frq"; + final static String POS_EXTENSION = "pos"; + final static String PAYLOAD_EXTENSION = "pyl"; + + // Increment version to change it: + final static int VERSION_START = 0; + final static int VERSION_CURRENT = VERSION_START; + + IntIndexOutput freqOut; + IntIndexOutput.Index freqIndex; + + IntIndexOutput posOut; + IntIndexOutput.Index posIndex; + + IntIndexOutput docOut; + IntIndexOutput.Index docIndex; + + IndexOutput payloadOut; + + IndexOutput skipOut; + + final SepSkipListWriter skipListWriter; + /** Expert: The fraction of TermDocs entries stored in skip tables, + * used to accelerate {@link DocsEnum#advance(int)}. Larger values result in + * smaller indexes, greater acceleration, but fewer accelerable cases, while + * smaller values result in bigger indexes, less acceleration and more + * accelerable cases. More detailed experiments would be useful here. */ + final int skipInterval; + static final int DEFAULT_SKIP_INTERVAL = 16; + + /** + * Expert: minimum docFreq to write any skip data at all + */ + final int skipMinimum; + + /** Expert: The maximum number of skip levels. Smaller values result in + * slightly smaller indexes, but slower skipping in big posting lists. + */ + final int maxSkipLevels = 10; + + final int totalNumDocs; + + boolean storePayloads; + IndexOptions indexOptions; + + FieldInfo fieldInfo; + + int lastPayloadLength; + int lastPosition; + long payloadStart; + int lastDocID; + int df; + + SepTermState lastState; + long lastPayloadFP; + long lastSkipFP; + + public TempSepPostingsWriter(SegmentWriteState state, IntStreamFactory factory) throws IOException { + this(state, factory, DEFAULT_SKIP_INTERVAL); + } + + public TempSepPostingsWriter(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException { + freqOut = null; + freqIndex = null; + posOut = null; + posIndex = null; + payloadOut = null; + boolean success = false; + try { + this.skipInterval = skipInterval; + this.skipMinimum = skipInterval; /* set to the same for now */ + final String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, DOC_EXTENSION); + + docOut = factory.createOutput(state.directory, docFileName, state.context); + docIndex = docOut.index(); + + if (state.fieldInfos.hasFreq()) { + final String frqFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FREQ_EXTENSION); + freqOut = factory.createOutput(state.directory, frqFileName, state.context); + freqIndex = freqOut.index(); + } + + if (state.fieldInfos.hasProx()) { + final String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, POS_EXTENSION); + posOut = factory.createOutput(state.directory, posFileName, state.context); + posIndex = posOut.index(); + + // TODO: -- only if at least one field stores payloads? + final String payloadFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, PAYLOAD_EXTENSION); + payloadOut = state.directory.createOutput(payloadFileName, state.context); + } + + final String skipFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SKIP_EXTENSION); + skipOut = state.directory.createOutput(skipFileName, state.context); + + totalNumDocs = state.segmentInfo.getDocCount(); + + skipListWriter = new SepSkipListWriter(skipInterval, + maxSkipLevels, + totalNumDocs, + freqOut, docOut, + posOut, payloadOut); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(docOut, skipOut, freqOut, posOut, payloadOut); + } + } + } + + @Override + public void init(IndexOutput termsOut) throws IOException { + CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT); + // TODO: -- just ask skipper to "start" here + termsOut.writeInt(skipInterval); // write skipInterval + termsOut.writeInt(maxSkipLevels); // write maxSkipLevels + termsOut.writeInt(skipMinimum); // write skipMinimum + } + + @Override + public SepTermState newTermState() { + return new SepTermState(); + } + + @Override + public void startTerm() throws IOException { + docIndex.mark(); + //System.out.println("SEPW: startTerm docIndex=" + docIndex); + + if (indexOptions != IndexOptions.DOCS_ONLY) { + freqIndex.mark(); + } + + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + posIndex.mark(); + payloadStart = payloadOut.getFilePointer(); + lastPayloadLength = -1; + } + + skipListWriter.resetSkip(docIndex, freqIndex, posIndex); + } + + // Currently, this instance is re-used across fields, so + // our parent calls setField whenever the field changes + @Override + public int setField(FieldInfo fieldInfo) { + this.fieldInfo = fieldInfo; + this.indexOptions = fieldInfo.getIndexOptions(); + if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) { + throw new UnsupportedOperationException("this codec cannot index offsets"); + } + skipListWriter.setIndexOptions(indexOptions); + storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.hasPayloads(); + lastPayloadFP = 0; + lastSkipFP = 0; + lastState = setEmptyState(); + return 0; + } + + private SepTermState setEmptyState() { + SepTermState emptyState = new SepTermState(); + emptyState.docIndex = docOut.index(); + if (indexOptions != IndexOptions.DOCS_ONLY) { + emptyState.freqIndex = freqOut.index(); + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + emptyState.posIndex = posOut.index(); + } + } + emptyState.payloadFP = 0; + emptyState.skipFP = 0; + return emptyState; + } + + /** Adds a new doc in this term. If this returns null + * then we just skip consuming positions/payloads. */ + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + + final int delta = docID - lastDocID; + //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut); + + if (docID < 0 || (df > 0 && delta <= 0)) { + throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")"); + } + + if ((++df % skipInterval) == 0) { + // TODO: -- awkward we have to make these two + // separate calls to skipper + //System.out.println(" buffer skip lastDocID=" + lastDocID); + skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength); + skipListWriter.bufferSkip(df); + } + + lastDocID = docID; + docOut.write(delta); + if (indexOptions != IndexOptions.DOCS_ONLY) { + //System.out.println(" sepw startDoc: write freq=" + termDocFreq); + freqOut.write(termDocFreq); + } + } + + /** Add a new position & payload */ + @Override + public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException { + assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; + + final int delta = position - lastPosition; + assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it) + lastPosition = position; + + if (storePayloads) { + final int payloadLength = payload == null ? 0 : payload.length; + if (payloadLength != lastPayloadLength) { + lastPayloadLength = payloadLength; + // TODO: explore whether we get better compression + // by not storing payloadLength into prox stream? + posOut.write((delta<<1)|1); + posOut.write(payloadLength); + } else { + posOut.write(delta << 1); + } + + if (payloadLength > 0) { + payloadOut.writeBytes(payload.bytes, payload.offset, payloadLength); + } + } else { + posOut.write(delta); + } + + lastPosition = position; + } + + /** Called when we are done adding positions & payloads */ + @Override + public void finishDoc() { + lastPosition = 0; + } + + private static class SepTermState extends BlockTermState { + public IntIndexOutput.Index docIndex; + public IntIndexOutput.Index freqIndex; + public IntIndexOutput.Index posIndex; + public long payloadFP; + public long skipFP; + } + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(BlockTermState _state) throws IOException { + SepTermState state = (SepTermState)_state; + // TODO: -- wasteful we are counting this in two places? + assert state.docFreq > 0; + assert state.docFreq == df; + + state.docIndex = docOut.index(); + state.docIndex.copyFrom(docIndex, false); + if (indexOptions != IndexOptions.DOCS_ONLY) { + state.freqIndex = freqOut.index(); + state.freqIndex.copyFrom(freqIndex, false); + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + state.posIndex = posOut.index(); + state.posIndex.copyFrom(posIndex, false); + } else { + state.posIndex = null; + } + } else { + state.freqIndex = null; + state.posIndex = null; + } + + if (df >= skipMinimum) { + state.skipFP = skipOut.getFilePointer(); + //System.out.println(" skipFP=" + skipFP); + skipListWriter.writeSkip(skipOut); + //System.out.println(" numBytes=" + (skipOut.getFilePointer()-skipFP)); + } else { + state.skipFP = -1; + } + state.payloadFP = payloadStart; + + lastDocID = 0; + df = 0; + } + + @Override + public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException { + SepTermState state = (SepTermState)_state; + if (absolute) { + lastSkipFP = 0; + lastPayloadFP = 0; + lastState = state; + } + lastState.docIndex.copyFrom(state.docIndex, false); + lastState.docIndex.write(out, absolute); + if (indexOptions != IndexOptions.DOCS_ONLY) { + lastState.freqIndex.copyFrom(state.freqIndex, false); + lastState.freqIndex.write(out, absolute); + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + lastState.posIndex.copyFrom(state.posIndex, false); + lastState.posIndex.write(out, absolute); + if (storePayloads) { + if (absolute) { + out.writeVLong(state.payloadFP); + } else { + out.writeVLong(state.payloadFP - lastPayloadFP); + } + lastPayloadFP = state.payloadFP; + } + } + } + if (state.skipFP != -1) { + if (absolute) { + out.writeVLong(state.skipFP); + } else { + out.writeVLong(state.skipFP - lastSkipFP); + } + lastSkipFP = state.skipFP; + } + } + + @Override + public void close() throws IOException { + IOUtils.close(docOut, skipOut, freqOut, posOut, payloadOut); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java index ca851727ecf..45e9da54165 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java @@ -74,7 +74,7 @@ import org.junit.Ignore; // we won't even be running the actual code, only the impostor // @SuppressCodecs("Lucene4x") // Sep codec cannot yet handle the offsets in our 4.x index! -@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41"}) +@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "TempSep", "TempFixedIntBlock", "TempVariableIntBlock"}) public class TestBackwardsCompatibility extends LuceneTestCase { // Uncomment these cases & run them on an older Lucene version, diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java b/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java index f25e80bf6d2..f9455f4b02b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java @@ -49,7 +49,7 @@ import org.apache.lucene.util._TestUtil; // TODO: we really need to test indexingoffsets, but then getting only docs / docs + freqs. // not all codecs store prx separate... // TODO: fix sep codec to index offsets so we can greatly reduce this list! -@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"}) +@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "TempSep", "TempFixedIntBlock", "TempVariableIntBlock"}) public class TestPostingsOffsets extends LuceneTestCase { IndexWriterConfig iwc; diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexOutput.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexOutput.java index a7fad99983c..c8aa4172e7f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexOutput.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexOutput.java @@ -18,6 +18,7 @@ package org.apache.lucene.codecs.mocksep; */ import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Directory; import org.apache.lucene.util.IOUtils; @@ -86,7 +87,7 @@ public class MockSingleIntIndexOutput extends IntIndexOutput { } } @Override - public void write(IndexOutput indexOut, boolean absolute) + public void write(DataOutput indexOut, boolean absolute) throws IOException { if (absolute) { indexOut.writeVLong(fp); diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/temp/TempFixedIntBlockPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/temp/TempFixedIntBlockPostingsFormat.java new file mode 100644 index 00000000000..34d9f7cabf9 --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/temp/TempFixedIntBlockPostingsFormat.java @@ -0,0 +1,198 @@ +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.IOException; + +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.TempPostingsReaderBase; +import org.apache.lucene.codecs.TempPostingsWriterBase; +import org.apache.lucene.codecs.blockterms.BlockTermsReader; +import org.apache.lucene.codecs.blockterms.BlockTermsWriter; +import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader; +import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter; +import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; +import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; +import org.apache.lucene.codecs.intblock.FixedIntBlockIndexInput; +import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput; +import org.apache.lucene.codecs.sep.*; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.*; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; + +/** + * A silly test codec to verify core support for fixed + * sized int block encoders is working. The int encoder + * used here just writes each block as a series of vInt. + */ + +public final class TempFixedIntBlockPostingsFormat extends PostingsFormat { + + private final int blockSize; + + public TempFixedIntBlockPostingsFormat() { + this(1); + } + + public TempFixedIntBlockPostingsFormat(int blockSize) { + super("TempFixedIntBlock"); + this.blockSize = blockSize; + } + + @Override + public String toString() { + return getName() + "(blockSize=" + blockSize + ")"; + } + + // only for testing + public IntStreamFactory getIntFactory() { + return new MockIntFactory(blockSize); + } + + /** + * Encodes blocks as vInts of a fixed block size. + */ + public static class MockIntFactory extends IntStreamFactory { + private final int blockSize; + + public MockIntFactory(int blockSize) { + this.blockSize = blockSize; + } + + @Override + public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException { + return new FixedIntBlockIndexInput(dir.openInput(fileName, context)) { + + @Override + protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) { + return new BlockReader() { + public void seek(long pos) {} + @Override + public void readBlock() throws IOException { + for(int i=0;i= count: "buffer.length=" + buffer.length + " count=" + count; + for(int i=0;i