LUCENE-2589: add support for variable sized int block codecs

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@983417 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2010-08-08 14:25:45 +00:00
parent ee9cd510cc
commit 63f30c694d
19 changed files with 708 additions and 215 deletions

View File

@ -548,6 +548,10 @@ New features
* LUCENE-2526: Don't throw NPE from MultiPhraseQuery.toString when * LUCENE-2526: Don't throw NPE from MultiPhraseQuery.toString when
it's empty. (Ross Woolf via Mike McCandless) it's empty. (Ross Woolf via Mike McCandless)
* LUCENE-2589: Add a VariableSizedIntIndexInput, which, when used w/
Sep*, makes it simple to take any variable sized int block coders
(like Simple9/16) and use them in a codec. (Mike McCandless)
Optimizations Optimizations

View File

@ -56,4 +56,9 @@ public abstract class Codec {
/** Records all file extensions this codec uses */ /** Records all file extensions this codec uses */
public abstract void getExtensions(Set<String> extensions); public abstract void getExtensions(Set<String> extensions);
@Override
public String toString() {
return name;
}
} }

View File

@ -115,7 +115,7 @@ class DefaultCodecProvider extends CodecProvider {
DefaultCodecProvider() { DefaultCodecProvider() {
register(new StandardCodec()); register(new StandardCodec());
register(new PreFlexCodec()); register(new PreFlexCodec());
register(new PulsingCodec()); register(new PulsingCodec(1));
} }
@Override @Override

View File

@ -37,10 +37,10 @@ import org.apache.lucene.util.IntsRef;
*/ */
public abstract class FixedIntBlockIndexInput extends IntIndexInput { public abstract class FixedIntBlockIndexInput extends IntIndexInput {
private IndexInput in; private final IndexInput in;
protected int blockSize; protected final int blockSize;
protected void init(final IndexInput in) throws IOException { public FixedIntBlockIndexInput(final IndexInput in) throws IOException {
this.in = in; this.in = in;
blockSize = in.readVInt(); blockSize = in.readVInt();
} }

View File

@ -36,19 +36,19 @@ import org.apache.lucene.store.IndexOutput;
*/ */
public abstract class FixedIntBlockIndexOutput extends IntIndexOutput { public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
private IndexOutput out; protected final IndexOutput out;
private int blockSize; private final int blockSize;
private int[] pending; protected final int[] buffer;
private int upto; private int upto;
protected void init(IndexOutput out, int fixedBlockSize) throws IOException { protected FixedIntBlockIndexOutput(IndexOutput out, int fixedBlockSize) throws IOException {
blockSize = fixedBlockSize; blockSize = fixedBlockSize;
out.writeVInt(blockSize);
this.out = out; this.out = out;
pending = new int[blockSize]; out.writeVInt(blockSize);
buffer = new int[blockSize];
} }
protected abstract void flushBlock(int[] buffer, IndexOutput out) throws IOException; protected abstract void flushBlock() throws IOException;
@Override @Override
public Index index() throws IOException { public Index index() throws IOException {
@ -96,9 +96,9 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
@Override @Override
public void write(int v) throws IOException { public void write(int v) throws IOException {
pending[upto++] = v; buffer[upto++] = v;
if (upto == blockSize) { if (upto == blockSize) {
flushBlock(pending, out); flushBlock();
upto = 0; upto = 0;
} }
} }
@ -107,9 +107,9 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
public void close() throws IOException { public void close() throws IOException {
try { try {
if (upto > 0) { if (upto > 0) {
// NOTE: entries in the block after current upto are // NOTE: entries in the block after current upto are
// invalid // invalid
flushBlock(pending, out); flushBlock();
} }
} finally { } finally {
out.close(); out.close();

View File

@ -0,0 +1,216 @@
package org.apache.lucene.index.codecs.intblock;
/**
* 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.
*/
/** Naive int block API that writes vInts. This is
* expected to give poor performance; it's really only for
* testing the pluggability. One should typically use pfor instead. */
import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IntsRef;
// TODO: much of this can be shared code w/ the fixed case
/** Abstract base class that reads variable-size blocks of ints
* from an IndexInput. While this is a simple approach, a
* more performant approach would directly create an impl
* of IntIndexInput inside Directory. Wrapping a generic
* IndexInput will likely cost performance.
*
* @lucene.experimental
*/
public abstract class VariableIntBlockIndexInput extends IntIndexInput {
protected final IndexInput in;
protected final int maxBlockSize;
protected VariableIntBlockIndexInput(final IndexInput in) throws IOException {
this.in = in;
maxBlockSize = in.readInt();
}
@Override
public Reader reader() throws IOException {
final int[] buffer = new int[maxBlockSize];
final IndexInput clone = (IndexInput) in.clone();
// TODO: can this be simplified?
return new Reader(clone, buffer, this.getBlockReader(clone, buffer));
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public Index index() {
return new Index();
}
protected abstract BlockReader getBlockReader(IndexInput in, int[] buffer) throws IOException;
public interface BlockReader {
public int readBlock() throws IOException;
public void seek(long pos) throws IOException;
}
public static class Reader extends IntIndexInput.Reader {
private final IndexInput in;
public final int[] pending;
int upto;
private boolean seekPending;
private long pendingFP;
private int pendingUpto;
private long lastBlockFP;
private int blockSize;
private final BlockReader blockReader;
private final IntsRef bulkResult = new IntsRef();
public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader)
throws IOException {
this.in = in;
this.pending = pending;
bulkResult.ints = pending;
this.blockReader = blockReader;
}
void seek(final long fp, final int upto) throws IOException {
// TODO: should we do this in real-time, not lazy?
pendingFP = fp;
pendingUpto = upto;
assert pendingUpto >= 0: "pendingUpto=" + pendingUpto;
seekPending = true;
}
private final void maybeSeek() throws IOException {
if (seekPending) {
if (pendingFP != lastBlockFP) {
// need new block
in.seek(pendingFP);
blockReader.seek(pendingFP);
lastBlockFP = pendingFP;
blockSize = blockReader.readBlock();
}
upto = pendingUpto;
// TODO: if we were more clever when writing the
// index, such that a seek point wouldn't be written
// until the int encoder "committed", we could avoid
// this (likely minor) inefficiency:
// This is necessary for int encoders that are
// non-causal, ie must see future int values to
// encode the current ones.
while(upto >= blockSize) {
upto -= blockSize;
lastBlockFP = in.getFilePointer();
blockSize = blockReader.readBlock();
}
seekPending = false;
}
}
@Override
public int next() throws IOException {
this.maybeSeek();
if (upto == blockSize) {
lastBlockFP = in.getFilePointer();
blockSize = blockReader.readBlock();
upto = 0;
}
return pending[upto++];
}
@Override
public IntsRef read(final int count) throws IOException {
this.maybeSeek();
if (upto == blockSize) {
lastBlockFP = in.getFilePointer();
blockSize = blockReader.readBlock();
upto = 0;
}
bulkResult.offset = upto;
if (upto + count < blockSize) {
bulkResult.length = count;
upto += count;
} else {
bulkResult.length = blockSize - upto;
upto = blockSize;
}
return bulkResult;
}
}
private class Index extends IntIndexInput.Index {
private long fp;
private int upto;
@Override
public void read(final IndexInput indexIn, final boolean absolute) throws IOException {
if (absolute) {
fp = indexIn.readVLong();
upto = indexIn.readByte()&0xFF;
} else {
final long delta = indexIn.readVLong();
if (delta == 0) {
// same block
upto = indexIn.readByte()&0xFF;
} else {
// new block
fp += delta;
upto = indexIn.readByte()&0xFF;
}
}
// TODO: we can't do this assert because non-causal
// int encoders can have upto over the buffer size
//assert upto < maxBlockSize: "upto=" + upto + " max=" + maxBlockSize;
}
@Override
public String toString() {
return "VarIntBlock.Index fp=" + fp + " upto=" + upto + " maxBlock=" + maxBlockSize;
}
@Override
public void seek(final IntIndexInput.Reader other) throws IOException {
((Reader) other).seek(fp, upto);
}
@Override
public void set(final IntIndexInput.Index other) {
final Index idx = (Index) other;
fp = idx.fp;
upto = idx.upto;
}
@Override
public Object clone() {
Index other = new Index();
other.fp = fp;
other.upto = upto;
return other;
}
}
}

View File

@ -0,0 +1,128 @@
package org.apache.lucene.index.codecs.intblock;
/**
* 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.
*/
/** Naive int block API that writes vInts. This is
* expected to give poor performance; it's really only for
* testing the pluggability. One should typically use pfor instead. */
import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.store.IndexOutput;
// TODO: much of this can be shared code w/ the fixed case
/** Abstract base class that writes variable-size blocks of ints
* to an IndexOutput. While this is a simple approach, a
* more performant approach would directly create an impl
* of IntIndexOutput inside Directory. Wrapping a generic
* IndexInput will likely cost performance.
*
* @lucene.experimental
*/
public abstract class VariableIntBlockIndexOutput extends IntIndexOutput {
protected final IndexOutput out;
private int upto;
private static final int MAX_BLOCK_SIZE = 1 << 8;
/** NOTE: maxBlockSize plus the max non-causal lookahead
* of your codec must be less than 256. EG Simple9
* requires lookahead=1 because on seeing the Nth value
* it knows it must now encode the N-1 values before it. */
protected VariableIntBlockIndexOutput(IndexOutput out, int maxBlockSize) throws IOException {
if (maxBlockSize > MAX_BLOCK_SIZE) {
throw new IllegalArgumentException("maxBlockSize must be <= " + MAX_BLOCK_SIZE + "; got " + maxBlockSize);
}
this.out = out;
out.writeInt(maxBlockSize);
}
/** Called one value at a time. Return the number of
* buffered input values that have been written to out. */
protected abstract int add(int value) throws IOException;
@Override
public Index index() throws IOException {
return new Index();
}
private class Index extends IntIndexOutput.Index {
long fp;
int upto;
long lastFP;
int lastUpto;
@Override
public void mark() throws IOException {
fp = out.getFilePointer();
upto = VariableIntBlockIndexOutput.this.upto;
}
@Override
public void set(IntIndexOutput.Index other) throws IOException {
Index idx = (Index) other;
lastFP = fp = idx.fp;
lastUpto = upto = idx.upto;
}
@Override
public void write(IndexOutput indexOut, boolean absolute) throws IOException {
assert upto >= 0;
if (absolute) {
indexOut.writeVLong(fp);
indexOut.writeByte((byte) upto);
} else if (fp == lastFP) {
// same block
indexOut.writeVLong(0);
assert upto >= lastUpto;
indexOut.writeByte((byte) upto);
} else {
// new block
indexOut.writeVLong(fp - lastFP);
indexOut.writeByte((byte) upto);
}
lastUpto = upto;
lastFP = fp;
}
}
@Override
public void write(int v) throws IOException {
upto -= add(v)-1;
assert upto >= 0;
}
@Override
public void close() throws IOException {
try {
// stuff 0s in until the "real" data is flushed:
int stuffed = 0;
while(upto > stuffed) {
upto -= add(0)-1;
assert upto >= 0;
stuffed += 1;
}
} finally {
out.close();
}
}
}

View File

@ -50,8 +50,18 @@ import org.apache.lucene.util.BytesRef;
public class PulsingCodec extends Codec { public class PulsingCodec extends Codec {
public PulsingCodec() { private final int freqCutoff;
/** Terms with freq <= freqCutoff are inlined into terms
* dict. */
public PulsingCodec(int freqCutoff) {
name = "Pulsing"; name = "Pulsing";
this.freqCutoff = freqCutoff;
}
@Override
public String toString() {
return name + "(freqCutoff=" + freqCutoff + ")";
} }
@Override @Override
@ -62,7 +72,6 @@ public class PulsingCodec extends Codec {
// Terms that have <= freqCutoff number of docs are // Terms that have <= freqCutoff number of docs are
// "pulsed" (inlined): // "pulsed" (inlined):
final int freqCutoff = 1;
StandardPostingsWriter pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter); StandardPostingsWriter pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
// Terms dict index // Terms dict index

View File

@ -169,6 +169,11 @@ public class SepPostingsReaderImpl extends StandardPostingsReader {
skipOffset = other.skipOffset; skipOffset = other.skipOffset;
payloadOffset = other.payloadOffset; payloadOffset = other.payloadOffset;
} }
@Override
public String toString() {
return "tis.fp=" + filePointer + " docFreq=" + docFreq + " ord=" + ord + " docIndex=" + docIndex;
}
} }
@Override @Override
@ -629,12 +634,10 @@ public class SepPostingsReaderImpl extends StandardPostingsReader {
// positions // positions
while (pendingPosCount > freq) { while (pendingPosCount > freq) {
final int code = posReader.next(); final int code = posReader.next();
if (storePayloads) { if (storePayloads && (code & 1) != 0) {
if ((code & 1) != 0) { // Payload length has changed
// Payload length has changed payloadLength = posReader.next();
payloadLength = posReader.next(); assert payloadLength >= 0;
assert payloadLength >= 0;
}
} }
pendingPosCount--; pendingPosCount--;
payloadPending = true; payloadPending = true;

View File

@ -33,7 +33,7 @@ import java.io.IOException;
public final class CodecUtil { public final class CodecUtil {
private final static int CODEC_MAGIC = 0x3fd76c17; private final static int CODEC_MAGIC = 0x3fd76c17;
public static void writeHeader(IndexOutput out, String codec, int version) public static IndexOutput writeHeader(IndexOutput out, String codec, int version)
throws IOException { throws IOException {
final long start = out.getFilePointer(); final long start = out.getFilePointer();
out.writeInt(CODEC_MAGIC); out.writeInt(CODEC_MAGIC);
@ -44,6 +44,8 @@ public final class CodecUtil {
if (out.getFilePointer()-start != codec.length()+9) { if (out.getFilePointer()-start != codec.length()+9) {
throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]"); throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
} }
return out;
} }
public static int headerLength(String codec) { public static int headerLength(String codec) {

View File

@ -268,7 +268,7 @@ public class TestCodecs extends MultiCodecTestCase {
final int NUM_TERMS = 100; final int NUM_TERMS = 100;
final TermData[] terms = new TermData[NUM_TERMS]; final TermData[] terms = new TermData[NUM_TERMS];
for(int i=0;i<NUM_TERMS;i++) { for(int i=0;i<NUM_TERMS;i++) {
final int[] docs = new int[] {1}; final int[] docs = new int[] {i};
final String text = Integer.toString(i, Character.MAX_RADIX); final String text = Integer.toString(i, Character.MAX_RADIX);
terms[i] = new TermData(text, docs, null); terms[i] = new TermData(text, docs, null);
} }
@ -288,10 +288,21 @@ public class TestCodecs extends MultiCodecTestCase {
final FieldsEnum fieldsEnum = reader.iterator(); final FieldsEnum fieldsEnum = reader.iterator();
assertNotNull(fieldsEnum.next()); assertNotNull(fieldsEnum.next());
final TermsEnum termsEnum = fieldsEnum.terms(); final TermsEnum termsEnum = fieldsEnum.terms();
DocsEnum docsEnum = null;
for(int i=0;i<NUM_TERMS;i++) { for(int i=0;i<NUM_TERMS;i++) {
final BytesRef term = termsEnum.next(); final BytesRef term = termsEnum.next();
assertNotNull(term); assertNotNull(term);
assertEquals(terms[i].text2, term.utf8ToString()); assertEquals(terms[i].text2, term.utf8ToString());
// do this twice to stress test the codec's reuse, ie,
// make sure it properly fully resets (rewinds) its
// internal state:
for(int iter=0;iter<2;iter++) {
docsEnum = termsEnum.docs(null, docsEnum);
assertEquals(terms[i].docs[0], docsEnum.nextDoc());
assertEquals(DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
}
} }
assertNull(termsEnum.next()); assertNull(termsEnum.next());

View File

@ -27,13 +27,15 @@ public class TestIntBlockCodec extends LuceneTestCase {
public void testSimpleIntBlocks() throws Exception { public void testSimpleIntBlocks() throws Exception {
Directory dir = new MockRAMDirectory(); Directory dir = new MockRAMDirectory();
IntIndexOutput out = new MockFixedIntBlockIndexOutput(dir, "test", 128); IntStreamFactory f = new MockFixedIntBlockCodec(128).getIntFactory();
IntIndexOutput out = f.createOutput(dir, "test");
for(int i=0;i<11777;i++) { for(int i=0;i<11777;i++) {
out.write(i); out.write(i);
} }
out.close(); out.close();
IntIndexInput in = new MockFixedIntBlockIndexInput(dir, "test", 128); IntIndexInput in = f.openInput(dir, "test");
IntIndexInput.Reader r = in.reader(); IntIndexInput.Reader r = in.reader();
for(int i=0;i<11777;i++) { for(int i=0;i<11777;i++) {
@ -47,11 +49,13 @@ public class TestIntBlockCodec extends LuceneTestCase {
public void testEmptySimpleIntBlocks() throws Exception { public void testEmptySimpleIntBlocks() throws Exception {
Directory dir = new MockRAMDirectory(); Directory dir = new MockRAMDirectory();
IntIndexOutput out = new MockFixedIntBlockIndexOutput(dir, "test", 128); IntStreamFactory f = new MockFixedIntBlockCodec(128).getIntFactory();
IntIndexOutput out = f.createOutput(dir, "test");
// write no ints // write no ints
out.close(); out.close();
IntIndexInput in = new MockFixedIntBlockIndexInput(dir, "test", 128); IntIndexInput in = f.openInput(dir, "test");
in.reader(); in.reader();
// read no ints // read no ints
in.close(); in.close();

View File

@ -26,8 +26,13 @@ import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl; import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl; import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader; import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter; import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
@ -37,24 +42,70 @@ import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader; import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter; import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.*;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
/** /**
* A silly codec that simply writes each block as a series * A silly test codec to verify core support for fixed
* of vInts. Don't use this (performance will be poor)! * sized int block encoders is working. The int encoder
* This is here just to test the core intblock codec * used here just writes each block as a series of vInt.
* classes.
*/ */
public class MockFixedIntBlockCodec extends Codec { public class MockFixedIntBlockCodec extends Codec {
public MockFixedIntBlockCodec() { private final int blockSize;
public MockFixedIntBlockCodec(int blockSize) {
this.blockSize = blockSize;
name = "MockFixedIntBlock"; name = "MockFixedIntBlock";
} }
@Override
public String toString() {
return name + "(blockSize=" + blockSize + ")";
}
// only for testing
public IntStreamFactory getIntFactory() {
return new MockIntFactory();
}
private class MockIntFactory extends IntStreamFactory {
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) {
@Override
protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
return new BlockReader() {
public void seek(long pos) {}
public void readBlock() throws IOException {
for(int i=0;i<buffer.length;i++) {
buffer[i] = in.readVInt();
}
}
};
}
};
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
@Override
protected void flushBlock() throws IOException {
for(int i=0;i<buffer.length;i++) {
out.writeVInt(buffer[i]);
}
}
};
}
}
@Override @Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockFixedIntBlockFactory(1024)); StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
boolean success = false; boolean success = false;
StandardTermsIndexWriter indexWriter; StandardTermsIndexWriter indexWriter;
@ -88,7 +139,7 @@ public class MockFixedIntBlockCodec extends Codec {
StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir, StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo, state.segmentInfo,
state.readBufferSize, state.readBufferSize,
new MockFixedIntBlockFactory(1024)); new MockIntFactory());
StandardTermsIndexReader indexReader; StandardTermsIndexReader indexReader;
boolean success = false; boolean success = false;

View File

@ -1,44 +0,0 @@
package org.apache.lucene.index.codecs.mockintblock;
/**
* 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.store.Directory;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import java.io.IOException;
/** Silly int factory that reads/writes block of ints by
* simply encoding each as vInt. Don't use this
* (performance will be poor)! This is here just to test
* the core intblock codec classes.*/
public class MockFixedIntBlockFactory extends IntStreamFactory {
private final int blockSize;
public MockFixedIntBlockFactory(int blockSize) {
this.blockSize = blockSize;
}
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
return new MockFixedIntBlockIndexInput(dir, fileName, readBufferSize);
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
return new MockFixedIntBlockIndexOutput(dir, fileName, blockSize);
}
}

View File

@ -1,65 +0,0 @@
package org.apache.lucene.index.codecs.mockintblock;
/**
* 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.
*/
/** Naive int block API that writes vInts. This is
* expected to give poor performance; it's really only for
* testing the pluggability. One should typically use pfor instead. */
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
import java.io.IOException;
/** Don't use this class!! It naively encodes ints one vInt
* at a time. Use it only for testing. */
public class MockFixedIntBlockIndexInput extends FixedIntBlockIndexInput {
public MockFixedIntBlockIndexInput(Directory dir, String fileName, int readBufferSize) throws IOException {
IndexInput in = dir.openInput(fileName, readBufferSize);
CodecUtil.checkHeader(in, MockFixedIntBlockIndexOutput.CODEC,
MockFixedIntBlockIndexOutput.VERSION_START, MockFixedIntBlockIndexOutput.VERSION_START);
init(in);
}
private static class BlockReader implements FixedIntBlockIndexInput.BlockReader {
private final IndexInput in;
private final int[] buffer;
public BlockReader(IndexInput in, int[] buffer) {
this.in = in;
this.buffer = buffer;
}
public void readBlock() throws IOException {
// silly impl
for(int i=0;i<buffer.length;i++) {
buffer[i] = in.readVInt();
}
}
}
@Override
protected BlockReader getBlockReader(IndexInput in, int[] buffer) {
return new BlockReader(in, buffer);
}
}

View File

@ -1,53 +0,0 @@
package org.apache.lucene.index.codecs.mockintblock;
/**
* 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.
*/
/** Naive int block API that writes vInts. This is
* expected to give poor performance; it's really only for
* testing the pluggability. One should typically use pfor instead. */
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
import java.io.IOException;
/** Don't use this class!! It naively encodes ints one vInt
* at a time. Use it only for testing. */
public class MockFixedIntBlockIndexOutput extends FixedIntBlockIndexOutput {
public final static String CODEC = "SIMPLE_INT_BLOCKS";
public final static int VERSION_START = 0;
public final static int VERSION_CURRENT = VERSION_START;
public MockFixedIntBlockIndexOutput(Directory dir, String fileName, int blockSize) throws IOException {
IndexOutput out = dir.createOutput(fileName);
CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
init(out, blockSize);
}
@Override
protected void flushBlock(int[] buffer, IndexOutput out) throws IOException {
// silly impl
for(int i=0;i<buffer.length;i++) {
out.writeVInt(buffer[i]);
}
}
}

View File

@ -0,0 +1,218 @@
package org.apache.lucene.index.codecs.mockintblock;
/**
* 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.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexReader;
import org.apache.lucene.index.codecs.standard.SimpleStandardTermsIndexWriter;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.standard.StandardTermsDictReader;
import org.apache.lucene.index.codecs.standard.StandardTermsDictWriter;
import org.apache.lucene.index.codecs.standard.StandardTermsIndexReader;
import org.apache.lucene.index.codecs.standard.StandardTermsIndexWriter;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
/**
* A silly test codec to verify core support for variable
* sized int block encoders is working. The int encoder
* used here writes baseBlockSize ints at once, if the first
* int is <= 3, else 2*baseBlockSize.
*/
public class MockVariableIntBlockCodec extends Codec {
private final int baseBlockSize;
public MockVariableIntBlockCodec(int baseBlockSize) {
name = "MockVariableIntBlock";
this.baseBlockSize = baseBlockSize;
}
@Override
public String toString() {
return name + "(baseBlockSize="+ baseBlockSize + ")";
}
private class MockIntFactory extends IntStreamFactory {
@Override
public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
final IndexInput in = dir.openInput(fileName, readBufferSize);
final int baseBlockSize = in.readInt();
return new VariableIntBlockIndexInput(in) {
@Override
protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
return new BlockReader() {
public void seek(long pos) {}
public int readBlock() throws IOException {
buffer[0] = in.readVInt();
final int count = buffer[0] <= 3 ? baseBlockSize-1 : 2*baseBlockSize-1;
assert buffer.length >= count: "buffer.length=" + buffer.length + " count=" + count;
for(int i=0;i<count;i++) {
buffer[i+1] = in.readVInt();
}
return 1+count;
}
};
}
};
}
@Override
public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
final IndexOutput out = dir.createOutput(fileName);
out.writeInt(baseBlockSize);
return new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
int pendingCount;
final int[] buffer = new int[2+2*baseBlockSize];
@Override
protected int add(int value) throws IOException {
buffer[pendingCount++] = value;
// silly variable block length int encoder: if
// first value <= 3, we write N vints at once;
// else, 2*N
final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
// intentionally be non-causal here:
if (pendingCount == flushAt+1) {
for(int i=0;i<flushAt;i++) {
out.writeVInt(buffer[i]);
}
buffer[0] = buffer[flushAt];
pendingCount = 1;
return flushAt;
} else {
return 0;
}
}
};
}
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
StandardPostingsWriter postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory());
boolean success = false;
StandardTermsIndexWriter indexWriter;
try {
indexWriter = new SimpleStandardTermsIndexWriter(state);
success = true;
} finally {
if (!success) {
postingsWriter.close();
}
}
success = false;
try {
FieldsConsumer ret = new StandardTermsDictWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
if (!success) {
try {
postingsWriter.close();
} finally {
indexWriter.close();
}
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
StandardPostingsReader postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo,
state.readBufferSize,
new MockIntFactory());
StandardTermsIndexReader indexReader;
boolean success = false;
try {
indexReader = new SimpleStandardTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
} finally {
if (!success) {
postingsReader.close();
}
}
success = false;
try {
FieldsProducer ret = new StandardTermsDictReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE);
success = true;
return ret;
} finally {
if (!success) {
try {
postingsReader.close();
} finally {
indexReader.close();
}
}
}
}
@Override
public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
SepPostingsReaderImpl.files(segmentInfo, files);
StandardTermsDictReader.files(dir, segmentInfo, files);
SimpleStandardTermsIndexReader.files(dir, segmentInfo, files);
}
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
StandardTermsDictReader.getExtensions(extensions);
SimpleStandardTermsIndexReader.getIndexExtensions(extensions);
}
}

View File

@ -32,6 +32,7 @@ import junit.framework.TestCase;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.FieldCache; import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry; import org.apache.lucene.search.FieldCache.CacheEntry;
@ -83,7 +84,7 @@ public abstract class LuceneTestCase extends TestCase {
private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null; private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
private String codec; private Codec codec;
/** Used to track if setUp and tearDown are called correctly from subclasses */ /** Used to track if setUp and tearDown are called correctly from subclasses */
private boolean setup; private boolean setup;
@ -307,9 +308,7 @@ public abstract class LuceneTestCase extends TestCase {
seed = null; seed = null;
super.runBare(); super.runBare();
} catch (Throwable e) { } catch (Throwable e) {
if (TEST_CODEC.equals("random")) { System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
}
if (seed != null) { if (seed != null) {
System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed); System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed);
} }

View File

@ -31,8 +31,10 @@ import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec; import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
import org.apache.lucene.index.codecs.preflex.PreFlexCodec; import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
import org.apache.lucene.index.codecs.mocksep.MockSepCodec; import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec; import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -154,9 +156,9 @@ public class LuceneTestCaseJ4 {
// saves default codec: we do this statically as many build indexes in @beforeClass // saves default codec: we do this statically as many build indexes in @beforeClass
private static String savedDefaultCodec; private static String savedDefaultCodec;
private static String codec; private static Codec codec;
private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock"}; private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock"};
private static void swapCodec(Codec c) { private static void swapCodec(Codec c) {
final CodecProvider cp = CodecProvider.getDefault(); final CodecProvider cp = CodecProvider.getDefault();
@ -172,7 +174,7 @@ public class LuceneTestCaseJ4 {
} }
// returns current default codec // returns current default codec
static String installTestCodecs() { static Codec installTestCodecs() {
final CodecProvider cp = CodecProvider.getDefault(); final CodecProvider cp = CodecProvider.getDefault();
savedDefaultCodec = CodecProvider.getDefaultCodec(); savedDefaultCodec = CodecProvider.getDefaultCodec();
@ -190,15 +192,18 @@ public class LuceneTestCaseJ4 {
} }
swapCodec(new MockSepCodec()); swapCodec(new MockSepCodec());
swapCodec(new MockFixedIntBlockCodec()); swapCodec(new PulsingCodec(_TestUtil.nextInt(seedRnd, 1, 20)));
swapCodec(new MockFixedIntBlockCodec(_TestUtil.nextInt(seedRnd, 1, 2000)));
// baseBlockSize cannot be over 127:
swapCodec(new MockVariableIntBlockCodec(_TestUtil.nextInt(seedRnd, 1, 127)));
return codec; return cp.lookup(codec);
} }
// returns current PreFlex codec // returns current PreFlex codec
static void removeTestCodecs(String codec) { static void removeTestCodecs(Codec codec) {
final CodecProvider cp = CodecProvider.getDefault(); final CodecProvider cp = CodecProvider.getDefault();
if (codec.equals("PreFlex")) { if (codec.name.equals("PreFlex")) {
final Codec preFlex = cp.lookup("PreFlex"); final Codec preFlex = cp.lookup("PreFlex");
if (preFlex != null) { if (preFlex != null) {
cp.unregister(preFlex); cp.unregister(preFlex);
@ -207,6 +212,8 @@ public class LuceneTestCaseJ4 {
} }
cp.unregister(cp.lookup("MockSep")); cp.unregister(cp.lookup("MockSep"));
cp.unregister(cp.lookup("MockFixedIntBlock")); cp.unregister(cp.lookup("MockFixedIntBlock"));
cp.unregister(cp.lookup("MockVariableIntBlock"));
swapCodec(new PulsingCodec(1));
CodecProvider.setDefaultCodec(savedDefaultCodec); CodecProvider.setDefaultCodec(savedDefaultCodec);
} }
@ -530,9 +537,7 @@ public class LuceneTestCaseJ4 {
System.out.println("NOTE: random static seed of testclass '" + getName() + "' was: " + staticSeed); System.out.println("NOTE: random static seed of testclass '" + getName() + "' was: " + staticSeed);
} }
if (TEST_CODEC.equals("random")) { System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
System.out.println("NOTE: random codec of testcase '" + getName() + "' was: " + codec);
}
if (seed != null) { if (seed != null) {
System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed); System.out.println("NOTE: random seed of testcase '" + getName() + "' was: " + seed);