LUCENE-3209: add MemoryCodec

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1137208 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2011-06-18 15:45:58 +00:00
parent b6d0c850aa
commit 89307c3795
14 changed files with 848 additions and 13 deletions

View File

@ -429,6 +429,11 @@ New features
(Mike McCandless, Simon Willnauer)
* LUCENE-3209: Added MemoryCodec, which stores all terms & postings in
RAM as an FST; this is good for primary-key fields if you frequently
need to lookup by that field or perform deletions against it, for
example in a near-real-time setting. (Mike McCandless)
Optimizations
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms

View File

@ -22,7 +22,6 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.Map.Entry;
/** Holds a set of codecs, keyed by name. You subclass
* this, instantiate it, and register your codecs, then
@ -44,7 +43,7 @@ public class CodecProvider {
private final Set<String> knownExtensions = new HashSet<String>();
public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText"};
public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "Memory"};
public synchronized void register(Codec codec) {
if (codec.name == null) {
@ -84,8 +83,9 @@ public class CodecProvider {
public synchronized Codec lookup(String name) {
final Codec codec = codecs.get(name);
if (codec == null)
if (codec == null) {
throw new IllegalArgumentException("required codec '" + name + "' not found");
}
return codec;
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index.codecs;
* limitations under the License.
*/
import org.apache.lucene.index.codecs.memory.MemoryCodec;
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
@ -45,5 +46,6 @@ public class CoreCodecProvider extends CodecProvider {
register(new PreFlexCodec());
register(new PulsingCodec(1));
register(new SimpleTextCodec());
register(new MemoryCodec());
}
}

View File

@ -0,0 +1,780 @@
package org.apache.lucene.index.codecs.memory;
/**
* 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.Comparator;
import java.util.Iterator;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.codecs.PostingsConsumer;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
// TODO: would be nice to somehow allow this to act like
// InstantiatedIndex, by never writing to disk; ie you write
// to this Codec in RAM only and then when you open a reader
// it pulls the FST directly from what you wrote w/o going
// to disk.
/** Stores terms & postings (docs, positions, payloads) in
* RAM, using an FST.
*
* <p>Note that this codec implements advance as a linear
* scan! This means if you store large fields in here,
* queries that rely on advance will (AND BooleanQuery,
* PhraseQuery) will be relatively slow!
*
* @lucene.experimental */
public class MemoryCodec extends Codec {
public MemoryCodec() {
name = "Memory";
}
private static final boolean VERBOSE = false;
private final static class TermsWriter extends TermsConsumer {
private final IndexOutput out;
private final FieldInfo field;
private final Builder<BytesRef> builder;
private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
private int termCount;
public TermsWriter(IndexOutput out, FieldInfo field) {
this.out = out;
this.field = field;
builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, outputs);
// The byte[] output we create can easily be > 255 bytes:
builder.setAllowArrayArcs(false);
}
private class PostingsWriter extends PostingsConsumer {
private int lastDocID;
private int lastPos;
private int lastPayloadLen;
// NOTE: not private so we don't pay access check at runtime:
int docCount;
RAMOutputStream buffer = new RAMOutputStream();
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
if (VERBOSE) System.out.println(" startDoc docID=" + docID + " freq=" + termDocFreq);
final int delta = docID - lastDocID;
assert docID == 0 || delta > 0;
lastDocID = docID;
docCount++;
if (field.omitTermFreqAndPositions) {
buffer.writeVInt(delta);
} else if (termDocFreq == 1) {
buffer.writeVInt((delta<<1) | 1);
} else {
buffer.writeVInt(delta<<1);
assert termDocFreq > 0;
buffer.writeVInt(termDocFreq);
}
lastPos = 0;
}
@Override
public void addPosition(int pos, BytesRef payload) throws IOException {
assert payload == null || field.storePayloads;
if (VERBOSE) System.out.println(" addPos pos=" + pos + " payload=" + payload);
final int delta = pos - lastPos;
assert delta >= 0;
lastPos = pos;
if (field.storePayloads) {
final int payloadLen = payload == null ? 0 : payload.length;
if (payloadLen != lastPayloadLen) {
lastPayloadLen = payloadLen;
buffer.writeVInt((delta<<1)|1);
buffer.writeVInt(payloadLen);
} else {
buffer.writeVInt(delta<<1);
}
if (payloadLen > 0) {
buffer.writeBytes(payload.bytes, payload.offset, payloadLen);
}
} else {
buffer.writeVInt(delta);
}
}
@Override
public void finishDoc() {
}
public PostingsWriter reset() {
assert buffer.getFilePointer() == 0;
lastDocID = 0;
docCount = 0;
lastPayloadLen = 0;
return this;
}
}
private final PostingsWriter postingsWriter = new PostingsWriter();
@Override
public PostingsConsumer startTerm(BytesRef text) {
if (VERBOSE) System.out.println(" startTerm term=" + text.utf8ToString());
return postingsWriter.reset();
}
private final RAMOutputStream buffer2 = new RAMOutputStream();
private final BytesRef spare = new BytesRef();
private byte[] finalBuffer = new byte[128];
@Override
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
assert postingsWriter.docCount == stats.docFreq;
assert buffer2.getFilePointer() == 0;
buffer2.writeVInt(stats.docFreq);
if (!field.omitTermFreqAndPositions) {
buffer2.writeVLong(stats.totalTermFreq-stats.docFreq);
}
int pos = (int) buffer2.getFilePointer();
buffer2.writeTo(finalBuffer, 0);
buffer2.reset();
final int totalBytes = pos + (int) postingsWriter.buffer.getFilePointer();
if (totalBytes > finalBuffer.length) {
finalBuffer = ArrayUtil.grow(finalBuffer, totalBytes);
}
postingsWriter.buffer.writeTo(finalBuffer, pos);
postingsWriter.buffer.reset();
spare.bytes = finalBuffer;
spare.length = totalBytes;
if (VERBOSE) {
System.out.println(" finishTerm term=" + text.utf8ToString() + " " + totalBytes + " bytes totalTF=" + stats.totalTermFreq);
for(int i=0;i<totalBytes;i++) {
System.out.println(" " + Integer.toHexString(finalBuffer[i]&0xFF));
}
}
builder.add(text, new BytesRef(spare));
termCount++;
}
@Override
public void finish(long sumTotalTermFreq) throws IOException {
if (termCount > 0) {
out.writeVInt(termCount);
out.writeVInt(field.number);
if (!field.omitTermFreqAndPositions) {
out.writeVLong(sumTotalTermFreq);
}
builder.finish().save(out);
if (VERBOSE) System.out.println("finish field=" + field.name + " fp=" + out.getFilePointer());
}
}
@Override
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
}
private static String EXTENSION = "ram";
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
final IndexOutput out = state.directory.createOutput(fileName);
return new FieldsConsumer() {
@Override
public TermsConsumer addField(FieldInfo field) {
if (VERBOSE) System.out.println("\naddField field=" + field.name);
return new TermsWriter(out, field);
}
@Override
public void close() throws IOException {
// EOF marker:
try {
out.writeVInt(0);
} finally {
out.close();
}
}
};
}
private final static class FSTDocsEnum extends DocsEnum {
private final boolean omitTFAP;
private final boolean storePayloads;
private byte[] buffer = new byte[16];
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
private Bits skipDocs;
private int docUpto;
private int docID;
private int freq;
private int payloadLen;
private int numDocs;
public FSTDocsEnum(boolean omitTFAP, boolean storePayloads) {
this.omitTFAP = omitTFAP;
this.storePayloads = storePayloads;
}
public boolean canReuse(boolean omitTFAP, boolean storePayloads) {
return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
}
public FSTDocsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
assert numDocs > 0;
if (buffer.length < bufferIn.length - bufferIn.offset) {
buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
}
in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
this.skipDocs = skipDocs;
docID = 0;
docUpto = 0;
payloadLen = 0;
this.numDocs = numDocs;
return this;
}
@Override
public int nextDoc() {
while(true) {
if (VERBOSE) System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
if (docUpto == numDocs) {
if (VERBOSE) {
System.out.println(" END");
}
return docID = NO_MORE_DOCS;
}
docUpto++;
if (omitTFAP) {
docID += in.readVInt();
freq = 1;
} else {
final int code = in.readVInt();
docID += code >>> 1;
if (VERBOSE) System.out.println(" docID=" + docID + " code=" + code);
if ((code & 1) != 0) {
freq = 1;
} else {
freq = in.readVInt();
assert freq > 0;
}
// Skip positions
for(int posUpto=0;posUpto<freq;posUpto++) {
if (!storePayloads) {
in.readVInt();
} else {
final int posCode = in.readVInt();
if ((posCode & 1) != 0) {
payloadLen = in.readVInt();
}
in.skipBytes(payloadLen);
}
}
}
if (skipDocs == null || !skipDocs.get(docID)) {
if (VERBOSE) System.out.println(" return docID=" + docID + " freq=" + freq);
return docID;
}
}
}
@Override
public int docID() {
return docID;
}
@Override
public int advance(int target) {
// TODO: we could make more efficient version, but, it
// should be rare that this will matter in practice
// since usually apps will not store "big" fields in
// this codec!
//System.out.println("advance start docID=" + docID + " target=" + target);
while(nextDoc() < target) {
}
return docID;
}
@Override
public int freq() {
return freq;
}
}
private final static class FSTDocsAndPositionsEnum extends DocsAndPositionsEnum {
private final boolean omitTFAP;
private final boolean storePayloads;
private byte[] buffer = new byte[16];
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
private Bits skipDocs;
private int docUpto;
private int docID;
private int freq;
private int numDocs;
private int posPending;
private int payloadLength;
private boolean payloadRetrieved;
private int pos;
private final BytesRef payload = new BytesRef();
public FSTDocsAndPositionsEnum(boolean omitTFAP, boolean storePayloads) {
this.omitTFAP = omitTFAP;
this.storePayloads = storePayloads;
}
public boolean canReuse(boolean omitTFAP, boolean storePayloads) {
return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
}
public FSTDocsAndPositionsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
assert numDocs > 0;
if (VERBOSE) {
System.out.println("D&P reset bytes this=" + this);
for(int i=bufferIn.offset;i<bufferIn.length;i++) {
System.out.println(" " + Integer.toHexString(bufferIn.bytes[i]&0xFF));
}
}
if (buffer.length < bufferIn.length - bufferIn.offset) {
buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
}
in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
this.skipDocs = skipDocs;
docID = 0;
docUpto = 0;
payload.bytes = buffer;
payloadLength = 0;
this.numDocs = numDocs;
posPending = 0;
payloadRetrieved = false;
return this;
}
@Override
public int nextDoc() {
while (posPending > 0) {
nextPosition();
}
while(true) {
if (VERBOSE) System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
if (docUpto == numDocs) {
if (VERBOSE) System.out.println(" END");
return docID = NO_MORE_DOCS;
}
docUpto++;
if (omitTFAP) {
docID += in.readVInt();
freq = 1;
} else {
final int code = in.readVInt();
docID += code >>> 1;
if ((code & 1) != 0) {
freq = 1;
} else {
freq = in.readVInt();
assert freq > 0;
}
}
if (skipDocs == null || !skipDocs.get(docID)) {
pos = 0;
posPending = freq;
if (VERBOSE) System.out.println(" return docID=" + docID + " freq=" + freq);
return docID;
}
// Skip positions
for(int posUpto=0;posUpto<freq;posUpto++) {
if (!storePayloads) {
in.readVInt();
} else {
final int code = in.readVInt();
if ((code & 1) != 0) {
payloadLength = in.readVInt();
if (VERBOSE) System.out.println(" new payloadLen=" + payloadLength);
}
in.skipBytes(payloadLength);
}
}
}
}
@Override
public int nextPosition() {
if (VERBOSE) System.out.println(" nextPos storePayloads=" + storePayloads);
assert posPending > 0;
posPending--;
if (!storePayloads) {
pos += in.readVInt();
} else {
final int code = in.readVInt();
pos += code >>> 1;
if ((code & 1) != 0) {
payloadLength = in.readVInt();
//System.out.println(" new payloadLen=" + payloadLength);
//} else {
//System.out.println(" same payloadLen=" + payloadLength);
}
payload.offset = in.getPosition();
in.skipBytes(payloadLength);
payload.length = payloadLength;
payloadRetrieved = false;
}
if (VERBOSE) System.out.println(" pos=" + pos + " payload=" + payload + " fp=" + in.getPosition());
return pos;
}
@Override
public BytesRef getPayload() {
payloadRetrieved = true;
return payload;
}
@Override
public boolean hasPayload() {
return !payloadRetrieved && payload.length > 0;
}
@Override
public int docID() {
return docID;
}
@Override
public int advance(int target) {
// TODO: we could make more efficient version, but, it
// should be rare that this will matter in practice
// since usually apps will not store "big" fields in
// this codec!
//System.out.println("advance target=" + target);
while(nextDoc() < target) {
}
//System.out.println(" return " + docID);
return docID;
}
@Override
public int freq() {
return freq;
}
}
private final static class FSTTermsEnum extends TermsEnum {
private final FieldInfo field;
private final BytesRefFSTEnum<BytesRef> fstEnum;
private final ByteArrayDataInput buffer = new ByteArrayDataInput(null);
private int docFreq;
private long totalTermFreq;
private BytesRefFSTEnum.InputOutput<BytesRef> current;
public FSTTermsEnum(FieldInfo field, FST<BytesRef> fst) {
this.field = field;
fstEnum = new BytesRefFSTEnum<BytesRef>(fst);
}
private void readTermStats() throws IOException {
buffer.reset(current.output.bytes, 0, current.output.length);
docFreq = buffer.readVInt();
if (!field.omitTermFreqAndPositions) {
totalTermFreq = docFreq + buffer.readVLong();
} else {
totalTermFreq = 0;
}
current.output.offset = buffer.getPosition();
if (VERBOSE) System.out.println(" df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
}
@Override
public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
if (VERBOSE) System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
current = fstEnum.seekCeil(text);
if (current == null) {
return SeekStatus.END;
} else {
if (VERBOSE) {
System.out.println(" got term=" + current.input.utf8ToString());
for(int i=0;i<current.output.length;i++) {
System.out.println(" " + Integer.toHexString(current.output.bytes[i]&0xFF));
}
}
readTermStats();
if (text.equals(current.input)) {
if (VERBOSE) System.out.println(" found!");
return SeekStatus.FOUND;
} else {
if (VERBOSE) System.out.println(" not found: " + current.input.utf8ToString());
return SeekStatus.NOT_FOUND;
}
}
}
@Override
public DocsEnum docs(Bits skipDocs, DocsEnum reuse) {
FSTDocsEnum docsEnum;
if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
} else {
docsEnum = (FSTDocsEnum) reuse;
if (!docsEnum.canReuse(field.omitTermFreqAndPositions, field.storePayloads)) {
docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
}
}
return docsEnum.reset(current.output, skipDocs, docFreq);
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) {
if (field.omitTermFreqAndPositions) {
return null;
}
FSTDocsAndPositionsEnum docsAndPositionsEnum;
if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) {
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.omitTermFreqAndPositions, field.storePayloads);
} else {
docsAndPositionsEnum = (FSTDocsAndPositionsEnum) reuse;
if (!docsAndPositionsEnum.canReuse(field.omitTermFreqAndPositions, field.storePayloads)) {
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.omitTermFreqAndPositions, field.storePayloads);
}
}
if (VERBOSE) System.out.println("D&P reset this=" + this);
return docsAndPositionsEnum.reset(current.output, skipDocs, docFreq);
}
@Override
public BytesRef term() {
return current.input;
}
@Override
public BytesRef next() throws IOException {
if (VERBOSE) System.out.println("te.next");
current = fstEnum.next();
if (current == null) {
if (VERBOSE) System.out.println(" END");
return null;
}
readTermStats();
if (VERBOSE) System.out.println(" term=" + field.name + ":" + current.input.utf8ToString());
return current.input;
}
@Override
public int docFreq() {
return docFreq;
}
@Override
public long totalTermFreq() {
return totalTermFreq;
}
@Override
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public SeekStatus seek(long ord) {
// NOTE: we could add this...
throw new UnsupportedOperationException();
}
@Override
public long ord() {
// NOTE: we could add this...
throw new UnsupportedOperationException();
}
}
private final static class TermsReader extends Terms {
private final long sumTotalTermFreq;
private FST<BytesRef> fst;
private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
private final FieldInfo field;
public TermsReader(FieldInfos fieldInfos, IndexInput in) throws IOException {
final int fieldNumber = in.readVInt();
field = fieldInfos.fieldInfo(fieldNumber);
if (!field.omitTermFreqAndPositions) {
sumTotalTermFreq = in.readVInt();
} else {
sumTotalTermFreq = 0;
}
fst = new FST<BytesRef>(in, outputs);
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public TermsEnum iterator() {
return new FSTTermsEnum(field, fst);
}
@Override
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
final IndexInput in = state.dir.openInput(fileName);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
try {
while(true) {
final int termCount = in.readVInt();
if (termCount == 0) {
break;
}
final TermsReader termsReader = new TermsReader(state.fieldInfos, in);
fields.put(termsReader.field.name, termsReader);
}
} finally {
in.close();
}
return new FieldsProducer() {
@Override
public FieldsEnum iterator() {
final Iterator<TermsReader> iter = fields.values().iterator();
return new FieldsEnum() {
private TermsReader current;
@Override
public String next() {
current = iter.next();
return current.field.name;
}
public TermsEnum terms() {
return current.iterator();
}
};
}
@Override
public Terms terms(String field) {
return fields.get(field);
}
@Override
public void loadTermsIndex(int indexDivisor) {
// no op
}
@Override
public void close() {
// Drop ref to FST:
for(TermsReader termsReader : fields.values()) {
termsReader.fst = null;
}
}
};
}
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, EXTENSION));
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
}
@Override
public void getExtensions(Set<String> extensions) {
extensions.add(EXTENSION);
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
}
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
}
}

View File

@ -67,6 +67,26 @@ public class RAMOutputStream extends IndexOutput {
}
}
/** Copy the current contents of this buffer to output
* byte array */
public void writeTo(byte[] bytes, int offset) throws IOException {
flush();
final long end = file.length;
long pos = 0;
int buffer = 0;
int bytesUpto = offset;
while (pos < end) {
int length = BUFFER_SIZE;
long nextPos = pos + length;
if (nextPos > end) { // at the last buffer
length = (int)(end - pos);
}
System.arraycopy(file.getBuffer(buffer++), 0, bytes, bytesUpto, length);
bytesUpto += length;
pos = nextPos;
}
}
/** Resets this to an empty file. */
public void reset() {
currentBuffer = null;

View File

@ -21,7 +21,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.FST.INPUT_TYPE;
import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
import java.io.IOException;
@ -136,6 +136,13 @@ public class Builder<T> {
return dedupHash == null ? 0 : fst.nodeCount;
}
/** Pass false to disable the array arc optimization
* while building the FST. This is necessary if
* encoding a single arc may take more than 255 bytes. */
public void setAllowArrayArcs(boolean b) {
fst.setAllowArrayArcs(b);
}
private CompiledNode compileNode(UnCompiledNode<T> n) throws IOException {
final int address;

View File

@ -104,6 +104,8 @@ public class FST<T> {
// If arc has this label then that arc is final/accepted
public static final int END_LABEL = -1;
private boolean allowArrayArcs = true;
public final static class Arc<T> {
public int label;
public T output;
@ -795,6 +797,10 @@ public class FST<T> {
public int getArcWithOutputCount() {
return arcWithOutputCount;
}
public void setAllowArrayArcs(boolean v) {
allowArrayArcs = v;
}
/**
* Nodes will be expanded if their depth (distance from the root node) is
@ -812,8 +818,9 @@ public class FST<T> {
* @see Builder.UnCompiledNode#depth
*/
private boolean shouldExpand(UnCompiledNode<T> node) {
return (node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) ||
node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP;
return allowArrayArcs &&
((node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) ||
node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
}
// Non-static: writes to FST's byte[]

View File

@ -52,6 +52,7 @@ import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.index.codecs.memory.MemoryCodec;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry;
@ -1452,6 +1453,7 @@ public abstract class LuceneTestCase extends Assert {
register(new PreFlexCodec());
register(new PulsingCodec(1));
register(new SimpleTextCodec());
register(new MemoryCodec());
Collections.shuffle(knownCodecs, random);
}

View File

@ -1013,6 +1013,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
HashSet<String> illegalCodecs = new HashSet<String>();
illegalCodecs.add("PreFlex");
illegalCodecs.add("SimpleText");
illegalCodecs.add("Memory");
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random)).setReaderTermsIndexDivisor(-1);

View File

@ -133,6 +133,7 @@ public class TestLazyProxSkipping extends LuceneTestCase {
public void testLazySkipping() throws IOException {
assumeFalse("This test cannot run with SimpleText codec", CodecProvider.getDefault().getFieldCodec(this.field).equals("SimpleText"));
assumeFalse("This test cannot run with Memory codec", CodecProvider.getDefault().getFieldCodec(this.field).equals("Memory"));
// test whether only the minimum amount of seeks()
// are performed
performTest(5);

View File

@ -17,7 +17,6 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.File;
import java.io.IOException;
import java.io.StringReader;
@ -65,6 +64,7 @@ public class TestLongPostings extends LuceneTestCase {
public void testLongPostings() throws Exception {
assumeFalse("Too slow with SimpleText codec", CodecProvider.getDefault().getFieldCodec("field").equals("SimpleText"));
assumeFalse("Too slow with Memory codec", CodecProvider.getDefault().getFieldCodec("field").equals("Memory"));
// Don't use _TestUtil.getTempDir so that we own the
// randomness (ie same seed will point to same dir):

View File

@ -96,7 +96,8 @@ public class TestNRTThreads extends LuceneTestCase {
final long t0 = System.currentTimeMillis();
if (CodecProvider.getDefault().getDefaultFieldCodec().equals("SimpleText")) {
final String defaultCodec = CodecProvider.getDefault().getDefaultFieldCodec();
if (defaultCodec.equals("SimpleText") || defaultCodec.equals("Memory")) {
// no
CodecProvider.getDefault().setDefaultFieldCodec("Standard");
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.*;
import org.apache.lucene.document.Field.Index;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.store.*;
import org.apache.lucene.util.*;
import org.junit.Test;
@ -31,12 +32,18 @@ public class TestRollingUpdates extends LuceneTestCase {
@Test
public void testRollingUpdates() throws Exception {
final Directory dir = newDirectory();
final MockDirectoryWrapper dir = newDirectory();
dir.setCheckIndexOnClose(false); // we use a custom codec provider
final LineFileDocs docs = new LineFileDocs(random);
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
final int SIZE = atLeast(20);
CodecProvider provider = CodecProvider.getDefault();
//provider.register(new MemoryCodec());
if (random.nextBoolean()) {
provider.setFieldCodec("docid", "Memory");
}
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setCodecProvider(provider));
final int SIZE = atLeast(TEST_NIGHTLY ? 100 : 20);
int id = 0;
IndexReader r = null;
final int numUpdates = (int) (SIZE * (2+random.nextDouble()));
@ -71,6 +78,7 @@ public class TestRollingUpdates extends LuceneTestCase {
w.close();
docs.close();
_TestUtil.checkIndex(dir);
dir.close();
}

View File

@ -980,7 +980,8 @@ public class TestFSTs extends LuceneTestCase {
// file, up until a time limit
public void testRealTerms() throws Exception {
if (CodecProvider.getDefault().getDefaultFieldCodec().equals("SimpleText")) {
final String defaultCodec = CodecProvider.getDefault().getDefaultFieldCodec();
if (defaultCodec.equals("SimpleText") || defaultCodec.equals("Memory")) {
// no
CodecProvider.getDefault().setDefaultFieldCodec("Standard");
}