diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 7a4d166f328..aba0d87c84a 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -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 diff --git a/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java b/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java index cfc9c458311..4236120563d 100644 --- a/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java +++ b/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java @@ -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 knownExtensions = new HashSet(); - 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; } diff --git a/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java b/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java index d6bb79b5ac0..6f3934b401e 100644 --- a/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java +++ b/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java @@ -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()); } } diff --git a/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java b/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java new file mode 100644 index 00000000000..8b103cf21b3 --- /dev/null +++ b/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java @@ -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. + * + *

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 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(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 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 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 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 fstEnum; + private final ByteArrayDataInput buffer = new ByteArrayDataInput(null); + + private int docFreq; + private long totalTermFreq; + private BytesRefFSTEnum.InputOutput current; + + public FSTTermsEnum(FieldInfo field, FST fst) { + this.field = field; + fstEnum = new BytesRefFSTEnum(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 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 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(in, outputs); + } + + @Override + public long getSumTotalTermFreq() { + return sumTotalTermFreq; + } + + @Override + public TermsEnum iterator() { + return new FSTTermsEnum(field, fst); + } + + @Override + public Comparator 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 fields = new TreeMap(); + + 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 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 files) throws IOException { + files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, EXTENSION)); + DefaultDocValuesConsumer.files(dir, segmentInfo, id, files); + } + + @Override + public void getExtensions(Set 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); + } +} diff --git a/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java b/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java index b07fe07e7fe..62a3848c408 100644 --- a/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java +++ b/lucene/src/java/org/apache/lucene/store/RAMOutputStream.java @@ -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; diff --git a/lucene/src/java/org/apache/lucene/util/fst/Builder.java b/lucene/src/java/org/apache/lucene/util/fst/Builder.java index 99378cfb2fa..84245a6d577 100644 --- a/lucene/src/java/org/apache/lucene/util/fst/Builder.java +++ b/lucene/src/java/org/apache/lucene/util/fst/Builder.java @@ -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 { 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 n) throws IOException { final int address; diff --git a/lucene/src/java/org/apache/lucene/util/fst/FST.java b/lucene/src/java/org/apache/lucene/util/fst/FST.java index e8f5c8cf569..a52b26e293b 100644 --- a/lucene/src/java/org/apache/lucene/util/fst/FST.java +++ b/lucene/src/java/org/apache/lucene/util/fst/FST.java @@ -104,6 +104,8 @@ public class FST { // 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 { public int label; public T output; @@ -795,6 +797,10 @@ public class FST { 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 { * @see Builder.UnCompiledNode#depth */ private boolean shouldExpand(UnCompiledNode 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[] diff --git a/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java b/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java index 876d9442382..6dd4059842e 100644 --- a/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java +++ b/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java @@ -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); } diff --git a/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java b/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java index 673782bf2bf..696a884fd98 100644 --- a/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java +++ b/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java @@ -1013,6 +1013,7 @@ public class TestIndexWriterReader extends LuceneTestCase { HashSet illegalCodecs = new HashSet(); illegalCodecs.add("PreFlex"); illegalCodecs.add("SimpleText"); + illegalCodecs.add("Memory"); IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setReaderTermsIndexDivisor(-1); diff --git a/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java b/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java index 331430e435c..0d208d28179 100755 --- a/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java +++ b/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java @@ -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); diff --git a/lucene/src/test/org/apache/lucene/index/TestLongPostings.java b/lucene/src/test/org/apache/lucene/index/TestLongPostings.java index cb3c1507a00..a0d691654fb 100644 --- a/lucene/src/test/org/apache/lucene/index/TestLongPostings.java +++ b/lucene/src/test/org/apache/lucene/index/TestLongPostings.java @@ -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): diff --git a/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java b/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java index 270fbe6a1d9..aec6c684461 100644 --- a/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java +++ b/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java @@ -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"); } diff --git a/lucene/src/test/org/apache/lucene/index/TestRollingUpdates.java b/lucene/src/test/org/apache/lucene/index/TestRollingUpdates.java index 8bf1510a7d0..6a557f6d390 100644 --- a/lucene/src/test/org/apache/lucene/index/TestRollingUpdates.java +++ b/lucene/src/test/org/apache/lucene/index/TestRollingUpdates.java @@ -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(); } diff --git a/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java index d549675c2ad..d4aacaacaa9 100644 --- a/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java +++ b/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java @@ -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"); }