LUCENE-3661: move deletes under codec

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1237245 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-01-29 11:33:46 +00:00
commit 36cc6e315c
34 changed files with 701 additions and 250 deletions

View File

@ -22,6 +22,7 @@ import java.util.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.util.NamedSPILoader;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
/**
@ -43,7 +44,11 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
return name;
}
/** Populates <code>files</code> with all filenames needed for
* the <code>info</code> segment.
*/
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
assert (dir instanceof CompoundFileDirectory) == false;
postingsFormat().files(dir, info, "", files);
storedFieldsFormat().files(dir, info, files);
termVectorsFormat().files(dir, info, files);
@ -54,6 +59,14 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
normsFormat().files(dir, info, files);
}
/** Populates <code>files</code> with any filenames that are
* stored outside of CFS for the <code>info</code> segment.
*/
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
liveDocsFormat().separateFiles(dir, info, files);
normsFormat().separateFiles(dir, info, files);
}
/** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat();
@ -75,6 +88,9 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes document normalization values */
public abstract NormsFormat normsFormat();
/** Encodes/decodes live docs */
public abstract LiveDocsFormat liveDocsFormat();
/** looks up a codec by name */
public static Codec forName(String name) {
return loader.lookup(name);

View File

@ -0,0 +1,41 @@
package org.apache.lucene.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
/** Format for live/deleted documents
* @lucene.experimental */
public abstract class LiveDocsFormat {
/** creates a new mutablebits, with all bits set, for the specified size */
public abstract MutableBits newLiveDocs(int size) throws IOException;
/** creates a new mutablebits of the same bits set and size of existing */
public abstract MutableBits newLiveDocs(Bits existing) throws IOException;
/** reads bits from a file */
public abstract Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException;
/** writes bits to a file */
public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException;
public abstract void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
}

View File

@ -78,6 +78,7 @@ public final class MappingMultiDocsEnum extends DocsEnum {
current = subs[upto].docsEnum;
currentBase = mergeState.docBase[reader];
currentMap = mergeState.docMaps[reader];
assert currentMap == null || currentMap.length == subs[upto].slice.length: "readerIndex=" + reader + " subs.len=" + subs.length + " len1=" + currentMap.length + " vs " + subs[upto].slice.length;
}
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.appending;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@ -28,6 +29,7 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
@ -50,6 +52,7 @@ public class AppendingCodec extends Codec {
private final TermVectorsFormat vectors = new Lucene40TermVectorsFormat();
private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
private final NormsFormat norms = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
@Override
public PostingsFormat postingsFormat() {
@ -85,4 +88,9 @@ public class AppendingCodec extends Codec {
public NormsFormat normsFormat() {
return norms;
}
@Override
public LiveDocsFormat liveDocsFormat() {
return liveDocs;
}
}

View File

@ -23,18 +23,23 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.MutableBits;
/**
* Supports the Lucene 3.x index format (readonly)
@ -47,7 +52,12 @@ public class Lucene3xCodec extends Codec {
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
// TODO: this should really be a different impl
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat() {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};
private final TermVectorsFormat vectorsFormat = new Lucene3xTermVectorsFormat();
@ -57,6 +67,14 @@ public class Lucene3xCodec extends Codec {
private final NormsFormat normsFormat = new Lucene3xNormsFormat();
// TODO: this should really be a different impl
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() {
@Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};
// 3.x doesn't support docvalues
private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
@Override
@ -107,4 +125,9 @@ public class Lucene3xCodec extends Codec {
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.util;
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -25,6 +25,8 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.MutableBits;
/** Optimized implementation of a vector of bits. This is more-or-less like
* java.util.BitSet, but also includes the following:
@ -37,7 +39,9 @@ import org.apache.lucene.store.IndexOutput;
*
* @lucene.internal
*/
public final class BitVector implements Cloneable, Bits {
// pkg-private: if this thing is generally useful then it can go back in .util,
// but the serialization must be here underneath the codec.
final class BitVector implements Cloneable, MutableBits {
private byte[] bits;
private int size;
@ -66,7 +70,7 @@ public final class BitVector implements Cloneable, Bits {
}
@Override
public Object clone() {
public BitVector clone() {
byte[] copyBits = new byte[bits.length];
System.arraycopy(bits, 0, copyBits, 0, bits.length);
BitVector clone = new BitVector(copyBits, size);

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene40;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@ -42,6 +43,8 @@ public class Lucene40Codec extends Codec {
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
@ -87,6 +90,11 @@ public class Lucene40Codec extends Codec {
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -0,0 +1,56 @@
package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import java.util.Set;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
public class Lucene40LiveDocsFormat extends LiveDocsFormat {
/** Extension of deletes */
static final String DELETES_EXTENSION = "del";
@Override
public MutableBits newLiveDocs(int size) throws IOException {
BitVector bitVector = new BitVector(size);
bitVector.invertAll();
return bitVector;
}
@Override
public MutableBits newLiveDocs(Bits existing) throws IOException {
final BitVector liveDocs = (BitVector) existing;
return liveDocs.clone();
}
@Override
public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException {
String filename = IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen());
final BitVector liveDocs = new BitVector(dir, filename, context);
assert liveDocs.count() == info.docCount - info.getDelCount();
assert liveDocs.length() == info.docCount;
return liveDocs;
}
@Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException {
String filename = IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen());
final BitVector liveDocs = (BitVector) bits;
assert liveDocs.count() == info.docCount - info.getDelCount();
assert liveDocs.length() == info.docCount;
liveDocs.write(dir, filename, context);
}
@Override
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
if (info.hasDeletions()) {
files.add(IndexFileNames.fileNameFromGeneration(info.name, DELETES_EXTENSION, info.getDelGen()));
}
}
}

View File

@ -109,7 +109,7 @@ public class Floats {
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
arrayTemplate = DocValuesArray.TEMPLATES.get(type);
assert size == 4 || size == 8;
assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
}
@Override

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.simpletext;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@ -43,6 +44,7 @@ public final class SimpleTextCodec extends Codec {
private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
// TODO: need a plain-text impl (using the above)
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
public SimpleTextCodec() {
super("SimpleText");
@ -82,4 +84,9 @@ public final class SimpleTextCodec extends Codec {
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public LiveDocsFormat liveDocsFormat() {
return liveDocs;
}
}

View File

@ -0,0 +1,185 @@
package org.apache.lucene.codecs.simpletext;
/**
* 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.BitSet;
import java.util.Set;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
/**
* reads/writes plaintext live docs
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
* @lucene.experimental
*/
public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
static final String LIVEDOCS_EXTENSION = "liv";
final static BytesRef SIZE = new BytesRef("size ");
final static BytesRef DOC = new BytesRef(" doc ");
final static BytesRef END = new BytesRef("END");
@Override
public MutableBits newLiveDocs(int size) throws IOException {
return new SimpleTextMutableBits(size);
}
@Override
public MutableBits newLiveDocs(Bits existing) throws IOException {
final SimpleTextBits bits = (SimpleTextBits) existing;
return new SimpleTextMutableBits((BitSet)bits.bits.clone(), bits.size);
}
@Override
public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException {
assert info.hasDeletions();
BytesRef scratch = new BytesRef();
CharsRef scratchUTF16 = new CharsRef();
String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen());
IndexInput in = null;
boolean success = false;
try {
in = dir.openInput(fileName, context);
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, SIZE);
int size = parseIntAt(scratch, SIZE.length, scratchUTF16);
BitSet bits = new BitSet(size);
SimpleTextUtil.readLine(in, scratch);
while (!scratch.equals(END)) {
assert StringHelper.startsWith(scratch, DOC);
int docid = parseIntAt(scratch, DOC.length, scratchUTF16);
bits.set(docid);
SimpleTextUtil.readLine(in, scratch);
}
success = true;
return new SimpleTextBits(bits, size);
} finally {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
}
private int parseIntAt(BytesRef bytes, int offset, CharsRef scratch) throws IOException {
UnicodeUtil.UTF8toUTF16(bytes.bytes, bytes.offset+offset, bytes.length-offset, scratch);
return ArrayUtil.parseInt(scratch.chars, 0, scratch.length);
}
@Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException {
BitSet set = ((SimpleTextBits) bits).bits;
int size = bits.length();
BytesRef scratch = new BytesRef();
String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen());
IndexOutput out = null;
boolean success = false;
try {
out = dir.createOutput(fileName, context);
SimpleTextUtil.write(out, SIZE);
SimpleTextUtil.write(out, Integer.toString(size), scratch);
SimpleTextUtil.writeNewline(out);
for (int i = set.nextSetBit(0); i >= 0; i=set.nextSetBit(i + 1)) {
SimpleTextUtil.write(out, DOC);
SimpleTextUtil.write(out, Integer.toString(i), scratch);
SimpleTextUtil.writeNewline(out);
}
SimpleTextUtil.write(out, END);
SimpleTextUtil.writeNewline(out);
success = true;
} finally {
if (success) {
IOUtils.close(out);
} else {
IOUtils.closeWhileHandlingException(out);
}
}
}
@Override
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
if (info.hasDeletions()) {
files.add(IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen()));
}
}
// read-only
static class SimpleTextBits implements Bits {
final BitSet bits;
final int size;
SimpleTextBits(BitSet bits, int size) {
this.bits = bits;
this.size = size;
}
@Override
public boolean get(int index) {
return bits.get(index);
}
@Override
public int length() {
return size;
}
}
// read-write
static class SimpleTextMutableBits extends SimpleTextBits implements MutableBits {
SimpleTextMutableBits(int size) {
this(new BitSet(size), size);
bits.set(0, size);
}
SimpleTextMutableBits(BitSet bits, int size) {
super(bits, size);
}
@Override
public void clear(int bit) {
bits.clear(bit);
}
}
}

View File

@ -147,7 +147,7 @@ class BufferedDeletesStream {
};
/** Resolves the buffered deleted Term/Query/docIDs, into
* actual deleted docIDs in the liveDocs BitVector for
* actual deleted docIDs in the liveDocs MutableBits for
* each SegmentReader. */
public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List<SegmentInfo> infos) throws IOException {
final long t0 = System.currentTimeMillis();
@ -206,7 +206,7 @@ class BufferedDeletesStream {
delIDX--;
} else if (packet != null && segGen == packet.delGen()) {
assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet";
assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet gen=" + segGen;
//System.out.println(" eq");
// Lock order: IW -> BD -> RP

View File

@ -174,8 +174,8 @@ public class CheckIndex {
/** True if this segment has pending deletions. */
public boolean hasDeletions;
/** Name of the current deletions file name. */
public String deletionsFileName;
/** Current deletions generation. */
public long deletionsGen;
/** Number of deleted documents. */
public int numDeleted;
@ -526,15 +526,14 @@ public class CheckIndex {
segInfoStat.docStoreCompoundFile = info.getDocStoreIsCompoundFile();
}
final String delFileName = info.getDelFileName();
if (delFileName == null){
if (info.hasDeletions()) {
msg(" no deletions");
segInfoStat.hasDeletions = false;
}
else{
msg(" has deletions [delFileName=" + delFileName + "]");
msg(" has deletions [delGen=" + info.getDelGen() + "]");
segInfoStat.hasDeletions = true;
segInfoStat.deletionsFileName = delFileName;
segInfoStat.deletionsGen = info.getDelGen();
}
if (infoStream != null)
infoStream.print(" test: open reader.........");

View File

@ -166,7 +166,9 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
} else {
readerShared[i] = false;
// Steal the ref returned by SegmentReader ctor:
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i], IOContext.READ);
assert infos.info(i).dir == newReaders[i].getSegmentInfo().dir;
assert infos.info(i).hasDeletions();
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i].core, IOContext.READ);
}
}
success = true;

View File

@ -30,12 +30,12 @@ import org.apache.lucene.search.similarities.SimilarityProvider;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.MutableBits;
public class DocumentsWriterPerThread {
@ -114,13 +114,15 @@ public class DocumentsWriterPerThread {
static class FlushedSegment {
final SegmentInfo segmentInfo;
final BufferedDeletes segmentDeletes;
final BitVector liveDocs;
final MutableBits liveDocs;
final int delCount;
private FlushedSegment(SegmentInfo segmentInfo,
BufferedDeletes segmentDeletes, BitVector liveDocs) {
BufferedDeletes segmentDeletes, MutableBits liveDocs, int delCount) {
this.segmentInfo = segmentInfo;
this.segmentDeletes = segmentDeletes;
this.liveDocs = liveDocs;
this.delCount = delCount;
}
}
@ -448,11 +450,11 @@ public class DocumentsWriterPerThread {
// happens when an exception is hit processing that
// doc, eg if analyzer has some problem w/ the text):
if (pendingDeletes.docIDs.size() > 0) {
flushState.liveDocs = new BitVector(numDocsInRAM);
flushState.liveDocs.invertAll();
flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
for(int delDocID : pendingDeletes.docIDs) {
flushState.liveDocs.clear(delDocID);
}
flushState.delCountOnFlush = pendingDeletes.docIDs.size();
pendingDeletes.bytesUsed.addAndGet(-pendingDeletes.docIDs.size() * BufferedDeletes.BYTES_PER_DEL_DOCID);
pendingDeletes.docIDs.clear();
}
@ -475,7 +477,7 @@ public class DocumentsWriterPerThread {
pendingDeletes.terms.clear();
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");
infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec());
@ -504,7 +506,7 @@ public class DocumentsWriterPerThread {
doAfterFlush();
success = true;
return new FlushedSegment(newSegment, segmentDeletes, flushState.liveDocs);
return new FlushedSegment(newSegment, segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
} finally {
if (!success) {
if (segment != null) {

View File

@ -28,7 +28,6 @@ import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
@ -461,11 +460,15 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
// Mark it deleted. TODO: we could also skip
// writing its postings; this would be
// deterministic (just for this Term's docs).
// TODO: can we do this reach-around in a cleaner way????
if (state.liveDocs == null) {
state.liveDocs = new BitVector(state.numDocs);
state.liveDocs.invertAll();
state.liveDocs = docState.docWriter.codec.liveDocsFormat().newLiveDocs(state.numDocs);
}
if (state.liveDocs.get(docID)) {
state.delCountOnFlush++;
state.liveDocs.clear(docID);
}
state.liveDocs.clear(docID);
}
totTF += termDocFreq;

View File

@ -57,9 +57,6 @@ public final class IndexFileNames {
/** Extension of compound file for doc store files*/
public static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
/** Extension of deletes */
public static final String DELETES_EXTENSION = "del";
/**
* This array contains all filename extensions used by
* Lucene's index files, with one exception, namely the
@ -70,7 +67,6 @@ public final class IndexFileNames {
public static final String INDEX_EXTENSIONS[] = new String[] {
COMPOUND_FILE_EXTENSION,
COMPOUND_FILE_ENTRIES_EXTENSION,
DELETES_EXTENSION,
GEN_EXTENSION,
COMPOUND_FILE_STORE_EXTENSION,
};

View File

@ -30,10 +30,10 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@ -48,10 +48,11 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.TwoPhaseCommit;
@ -416,7 +417,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// docs, and it's copy-on-write (cloned whenever we need
// to change it but it's been shared to an external NRT
// reader).
public BitVector liveDocs;
public Bits liveDocs;
// How many further deletions we've done against
// liveDocs vs when we loaded it or last wrote it:
@ -446,6 +447,24 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
return rc > myRefCounts;
}
// Call only from assert!
public synchronized boolean verifyDocCounts() {
int count;
if (liveDocs != null) {
count = 0;
for(int docID=0;docID<info.docCount;docID++) {
if (liveDocs.get(docID)) {
count++;
}
}
} else {
count = info.docCount;
}
assert info.docCount - info.getDelCount() - pendingDeleteCount == count: "info.docCount=" + info.docCount + " info.getDelCount()=" + info.getDelCount() + " pendingDeleteCount=" + pendingDeleteCount + " count=" + count;;
return true;
}
// Returns true if any reader remains
public synchronized boolean removeReader(SegmentReader sr, boolean drop) throws IOException {
if (sr == reader) {
@ -468,17 +487,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
return reader != null || mergeReader != null;
}
// Called only from assert
private boolean countsMatch() {
if (liveDocs == null) {
assert pendingDeleteCount == 0;
} else {
assert liveDocs.count() == info.docCount - info.getDelCount() - pendingDeleteCount :
"liveDocs.count()=" + liveDocs.count() + " info.docCount=" + info.docCount + " info.delCount=" + info.getDelCount() + " pendingDelCount=" + pendingDeleteCount;
}
return true;
}
// Get reader for searching/deleting
public synchronized SegmentReader getReader(IOContext context) throws IOException {
//System.out.println(" livedocs=" + rld.liveDocs);
@ -486,7 +494,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (reader == null) {
reader = new SegmentReader(info, config.getReaderTermsIndexDivisor(), context);
if (liveDocs == null) {
liveDocs = (BitVector) reader.getLiveDocs();
liveDocs = reader.getLiveDocs();
}
//System.out.println("ADD seg=" + rld.info + " isMerge=" + isMerge + " " + readerMap.size() + " in pool");
}
@ -513,7 +521,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} else {
mergeReader = new SegmentReader(info, -1, context);
if (liveDocs == null) {
liveDocs = (BitVector) mergeReader.getLiveDocs();
liveDocs = mergeReader.getLiveDocs();
}
}
}
@ -526,8 +534,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
public synchronized boolean delete(int docID) {
assert liveDocs != null;
assert docID >= 0 && docID < liveDocs.length();
final boolean didDelete = liveDocs.getAndClear(docID);
assert !shared;
final boolean didDelete = liveDocs.get(docID);
if (didDelete) {
((MutableBits) liveDocs).clear(docID);
pendingDeleteCount++;
//System.out.println(" new del seg=" + info + " docID=" + docID + " pendingDelCount=" + pendingDeleteCount + " totDelCount=" + (info.docCount-liveDocs.count()));
}
@ -557,17 +567,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
getReader(context).decRef();
assert reader != null;
}
assert countsMatch();
shared = true;
if (liveDocs != null) {
return new SegmentReader(reader, liveDocs, info.docCount - info.getDelCount() - pendingDeleteCount);
return new SegmentReader(reader.getSegmentInfo(), reader.core, liveDocs, info.docCount - info.getDelCount() - pendingDeleteCount);
} else {
reader.incRef();
return reader;
}
}
public synchronized void initWritableLiveDocs() {
public synchronized void initWritableLiveDocs() throws IOException {
assert Thread.holdsLock(IndexWriter.this);
//System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
if (shared) {
@ -575,12 +584,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// SegmentReader sharing the current liveDocs
// instance; must now make a private clone so we can
// change it:
LiveDocsFormat liveDocsFormat = info.getCodec().liveDocsFormat();
if (liveDocs == null) {
//System.out.println("create BV seg=" + info);
liveDocs = new BitVector(info.docCount);
liveDocs.setAll();
liveDocs = liveDocsFormat.newLiveDocs(info.docCount);
} else {
liveDocs = (BitVector) liveDocs.clone();
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
}
shared = false;
} else {
@ -588,11 +597,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
public synchronized BitVector getReadOnlyLiveDocs() {
public synchronized Bits getReadOnlyLiveDocs() {
//System.out.println("getROLiveDocs seg=" + info);
assert Thread.holdsLock(IndexWriter.this);
shared = true;
assert countsMatch();
//if (liveDocs != null) {
//System.out.println(" liveCount=" + liveDocs.count());
//}
@ -611,29 +619,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Save in case we need to rollback on failure:
final SegmentInfo sav = (SegmentInfo) info.clone();
info.advanceDelGen();
info.setDelCount(info.getDelCount() + pendingDeleteCount);
// We can write directly to the actual name (vs to a
// .tmp & renaming it) because the file is not live
// until segments file is written:
final String delFileName = info.getDelFileName();
boolean success = false;
try {
liveDocs.write(dir, delFileName, IOContext.DEFAULT);
info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, dir, info, IOContext.DEFAULT);
success = true;
} finally {
if (!success) {
info.reset(sav);
try {
dir.deleteFile(delFileName);
} catch (Throwable t) {
// Suppress this so we keep throwing the
// original exception
}
}
}
assert (info.docCount - liveDocs.count()) == info.getDelCount() + pendingDeleteCount:
"delete count mismatch during commit: seg=" + info + " info.delCount=" + info.getDelCount() + " vs BitVector=" + (info.docCount-liveDocs.count() + " pendingDelCount=" + pendingDeleteCount);
info.setDelCount(info.getDelCount() + pendingDeleteCount);
pendingDeleteCount = 0;
return true;
} else {
@ -2205,7 +2204,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
/**
* Prepares the {@link SegmentInfo} for the new flushed segment and persists
* the deleted documents {@link BitVector}. Use
* the deleted documents {@link MutableBits}. Use
* {@link #publishFlushedSegment(SegmentInfo, FrozenBufferedDeletes)} to
* publish the returned {@link SegmentInfo} together with its segment private
* delete packet.
@ -2252,33 +2251,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Must write deleted docs after the CFS so we don't
// slurp the del file into CFS:
if (flushedSegment.liveDocs != null) {
final int delCount = flushedSegment.segmentInfo.docCount - flushedSegment.liveDocs.count();
final int delCount = flushedSegment.delCount;
assert delCount > 0;
newSegment.setDelCount(delCount);
newSegment.advanceDelGen();
final String delFileName = newSegment.getDelFileName();
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush: write " + delCount + " deletes to " + delFileName);
}
boolean success2 = false;
try {
// TODO: in the NRT case it'd be better to hand
// this del vector over to the
// shortly-to-be-opened SegmentReader and let it
// carry the changes; there's no reason to use
// filesystem as intermediary here.
flushedSegment.liveDocs.write(directory, delFileName, context);
success2 = true;
} finally {
if (!success2) {
try {
directory.deleteFile(delFileName);
} catch (Throwable t) {
// suppress this so we keep throwing the
// original exception
}
}
infoStream.message("IW", "flush: write " + delCount + " deletes gen=" + flushedSegment.segmentInfo.getDelGen());
}
// TODO: in the NRT case it'd be better to hand
// this del vector over to the
// shortly-to-be-opened SegmentReader and let it
// carry the changes; there's no reason to use
// filesystem as intermediary here.
SegmentInfo info = flushedSegment.segmentInfo;
Codec codec = info.getCodec();
codec.liveDocsFormat().writeLiveDocs(flushedSegment.liveDocs, directory, info, context);
}
success = true;
@ -3032,8 +3021,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
final int docCount = info.docCount;
final BitVector prevLiveDocs = merge.readerLiveDocs.get(i);
final BitVector currentLiveDocs;
final Bits prevLiveDocs = merge.readerLiveDocs.get(i);
final Bits currentLiveDocs;
ReadersAndLiveDocs rld = readerPool.get(info, false);
// We enrolled in mergeInit:
assert rld != null;
@ -3052,7 +3041,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// newly flushed deletes but mapping them to the new
// docIDs.
if (currentLiveDocs.count() < prevLiveDocs.count()) {
// Since we copy-on-write, if any new deletes were
// applied after merging has started, we can just
// check if the before/after liveDocs have changed.
// If so, we must carefully merge the liveDocs one
// doc at a time:
if (currentLiveDocs != prevLiveDocs) {
// This means this segment received new deletes
// since we started the merge, so we
// must merge them:
@ -3071,8 +3066,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
} else {
assert currentLiveDocs.count() == prevLiveDocs.count(): "currentLiveDocs.count()==" + currentLiveDocs.count() + " vs prevLiveDocs.count()=" + prevLiveDocs.count() + " info=" + info;
docUpto += currentLiveDocs.count();
docUpto += info.docCount - info.getDelCount() - rld.pendingDeleteCount;
}
} else if (currentLiveDocs != null) {
// This segment had no deletes before but now it
@ -3576,13 +3570,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
merge.readers = new ArrayList<SegmentReader>();
merge.readerLiveDocs = new ArrayList<BitVector>();
merge.readerLiveDocs = new ArrayList<Bits>();
// This is try/finally to make sure merger's readers are
// closed:
boolean success = false;
try {
int totDocCount = 0;
int segUpto = 0;
while(segUpto < sourceSegments.size()) {
@ -3595,13 +3588,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
assert reader != null;
// Carefully pull the most recent live docs:
final BitVector liveDocs;
final Bits liveDocs;
synchronized(this) {
// Must sync to ensure BufferedDeletesStream
// cannot change liveDocs/pendingDeleteCount while
// we pull a copy:
liveDocs = rld.getReadOnlyLiveDocs();
assert rld.verifyDocCounts();
if (infoStream.isEnabled("IW")) {
if (rld.pendingDeleteCount != 0) {
infoStream.message("IW", "seg=" + info + " delCount=" + info.getDelCount() + " pendingDelCount=" + rld.pendingDeleteCount);
@ -3612,23 +3607,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
}
merge.readerLiveDocs.add(liveDocs);
merge.readers.add(reader);
if (liveDocs == null || liveDocs.count() > 0) {
final int delCount = rld.pendingDeleteCount + info.getDelCount();
assert delCount <= info.docCount;
if (delCount < info.docCount) {
merger.add(reader, liveDocs);
totDocCount += liveDocs == null ? reader.maxDoc() : liveDocs.count();
} else {
//System.out.println(" skip seg: fully deleted");
}
segUpto++;
}
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge: total " + totDocCount + " docs");
}
merge.checkAborted(directory);
// This is where all the work happens:
@ -3639,11 +3627,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
merge.info.setCodec(codec);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge codec=" + codec);
infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount);
}
assert mergedDocCount == totDocCount: "mergedDocCount=" + mergedDocCount + " vs " + totDocCount;
// Very important to do this before opening the reader
// because codec must know if prox was written for
// this segment:
@ -4089,11 +4075,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
try {
assert assertNoSeparateFiles(files, directory, info);
for (String file : files) {
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
: ".del file is not allowed in .cfs: " + file;
assert !isSeparateNormsFile(file)
: "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
directory.copy(cfsDir, file, file, context);
checkAbort.work(directory.fileLength(file));
}
@ -4106,15 +4089,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
/**
* Returns true if the given filename ends with the separate norms file
* pattern: {@code SEPARATE_NORMS_EXTENSION + "[0-9]+"}.
* @deprecated only for asserting
* used only by assert: checks that filenames about to be put in cfs belong.
*/
@Deprecated
private static boolean isSeparateNormsFile(String filename) {
int idx = filename.lastIndexOf('.');
if (idx == -1) return false;
String ext = filename.substring(idx + 1);
return Pattern.matches("s[0-9]+", ext);
private static boolean assertNoSeparateFiles(Collection<String> files,
Directory dir, SegmentInfo info) throws IOException {
// maybe this is overkill, but codec naming clashes would be bad.
Set<String> separateFiles = new HashSet<String>();
info.getCodec().separateFiles(dir, info, separateFiles);
for (String file : files) {
assert !separateFiles.contains(file) : file + " should not go in CFS!";
}
return true;
}
}

View File

@ -24,7 +24,7 @@ import java.util.Map;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.SetOnce.AlreadySetException;
import org.apache.lucene.util.SetOnce;
@ -74,7 +74,7 @@ public abstract class MergePolicy implements java.io.Closeable {
int maxNumSegments = -1; // used by IndexWriter
public long estimatedMergeBytes; // used by IndexWriter
List<SegmentReader> readers; // used by IndexWriter
List<BitVector> readerLiveDocs; // used by IndexWriter
List<Bits> readerLiveDocs; // used by IndexWriter
public final List<SegmentInfo> segments;
public final int totalDocCount;
boolean aborted;

View File

@ -41,10 +41,10 @@ public class MergeState {
}
public FieldInfos fieldInfos;
public List<IndexReaderAndLiveDocs> readers; // Readers & liveDocs being merged
public int[][] docMaps; // Maps docIDs around deletions
public int[] docBase; // New docID base per reader
public int mergedDocCount; // Total # merged docs
public List<IndexReaderAndLiveDocs> readers; // Readers & liveDocs being merged
public int[][] docMaps; // Maps docIDs around deletions
public int[] docBase; // New docID base per reader
public int mergedDocCount; // Total # merged docs
public CheckAbort checkAbort;
public InfoStream infoStream;

View File

@ -326,16 +326,6 @@ public final class SegmentInfo implements Cloneable {
return si;
}
public String getDelFileName() {
if (delGen == NO) {
// In this case we know there is no deletion filename
// against this segment
return null;
} else {
return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.DELETES_EXTENSION, delGen);
}
}
/**
* @deprecated separate norms are not supported in >= 4.0
*/
@ -494,6 +484,9 @@ public final class SegmentInfo implements Cloneable {
} else {
codec.files(dir, this, fileSet);
}
// regardless of compound file setting: these files are always in the directory
codec.separateFiles(dir, this, fileSet);
if (docStoreOffset != -1) {
// We are sharing doc stores (stored fields, term
@ -505,18 +498,6 @@ public final class SegmentInfo implements Cloneable {
}
}
String delFileName = IndexFileNames.fileNameFromGeneration(name, IndexFileNames.DELETES_EXTENSION, delGen);
if (delFileName != null && (delGen >= YES || dir.fileExists(delFileName))) {
fileSet.add(delFileName);
}
// because separate norm files are unconditionally stored outside cfs,
// we must explicitly ask for their filenames if we might have separate norms:
// remove this when 3.x indexes are no longer supported
if (normGen != null) {
codec.normsFormat().separateFiles(dir, this, fileSet);
}
files = new ArrayList<String>(fileSet);
return files;

View File

@ -104,16 +104,12 @@ final class SegmentMerger {
// IndexWriter.close(false) takes to actually stop the
// threads.
final int numReaders = mergeState.readers.size();
// Remap docIDs
mergeState.docMaps = new int[numReaders][];
mergeState.docBase = new int[numReaders];
mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
mergeState.mergedDocCount = setDocMaps();
mergeFieldInfos();
setMatchingSegmentReaders();
mergeState.mergedDocCount = mergeFields();
int numMerged = mergeFields();
assert numMerged == mergeState.mergedDocCount;
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, segment, mergeState.fieldInfos, mergeState.mergedDocCount, termIndexInterval, codec, null, context);
mergeTerms(segmentWriteState);
@ -124,7 +120,7 @@ final class SegmentMerger {
}
if (mergeState.fieldInfos.hasVectors()) {
int numMerged = mergeVectors();
numMerged = mergeVectors();
assert numMerged == mergeState.mergedDocCount;
}
@ -283,37 +279,31 @@ final class SegmentMerger {
}
}
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
int docBase = 0;
final List<Fields> fields = new ArrayList<Fields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
for(MergeState.IndexReaderAndLiveDocs r : mergeState.readers) {
final Fields f = r.reader.fields();
final int maxDoc = r.reader.maxDoc();
if (f != null) {
slices.add(new ReaderUtil.Slice(docBase, maxDoc, fields.size()));
fields.add(f);
}
docBase += maxDoc;
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private int setDocMaps() throws IOException {
final int numReaders = mergeState.readers.size();
docBase = 0;
// Remap docIDs
mergeState.docMaps = new int[numReaders][];
mergeState.docBase = new int[numReaders];
mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
for(int i=0;i<numReaders;i++) {
int docBase = 0;
int i = 0;
while(i < mergeState.readers.size()) {
final MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
mergeState.docBase[i] = docBase;
final int maxDoc = reader.reader.maxDoc();
if (reader.liveDocs != null) {
final int docCount;
final Bits liveDocs = reader.liveDocs;
final int[] docMap;
if (liveDocs != null) {
int delCount = 0;
final Bits liveDocs = reader.liveDocs;
assert liveDocs != null;
final int[] docMap = mergeState.docMaps[i] = new int[maxDoc];
docMap = new int[maxDoc];
int newDocID = 0;
for(int j=0;j<maxDoc;j++) {
if (!liveDocs.get(j)) {
@ -323,14 +313,41 @@ final class SegmentMerger {
docMap[j] = newDocID++;
}
}
docBase += maxDoc - delCount;
docCount = maxDoc - delCount;
} else {
docBase += maxDoc;
docCount = maxDoc;
docMap = null;
}
mergeState.docMaps[i] = docMap;
docBase += docCount;
if (mergeState.payloadProcessorProvider != null) {
mergeState.dirPayloadProcessor[i] = mergeState.payloadProcessorProvider.getDirProcessor(reader.reader.directory());
}
i++;
}
return docBase;
}
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
final List<Fields> fields = new ArrayList<Fields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
int docBase = 0;
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
final MergeState.IndexReaderAndLiveDocs r = mergeState.readers.get(readerIndex);
final Fields f = r.reader.fields();
final int maxDoc = r.reader.maxDoc();
if (f != null) {
slices.add(new ReaderUtil.Slice(docBase, maxDoc, readerIndex));
fields.add(f);
}
docBase += maxDoc;
}
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);

View File

@ -25,7 +25,6 @@ import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.FieldCache; // javadocs
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
/**
@ -36,14 +35,14 @@ public final class SegmentReader extends IndexReader {
private final SegmentInfo si;
private final ReaderContext readerContext = new AtomicReaderContext(this);
private final BitVector liveDocs;
private final Bits liveDocs;
// Normally set to si.docCount - si.delDocCount, unless we
// were created as an NRT reader from IW, in which case IW
// tells us the docCount:
private final int numDocs;
private final SegmentCoreReaders core;
final SegmentCoreReaders core;
/**
* @throws CorruptIndexException if the index is corrupt
@ -56,13 +55,12 @@ public final class SegmentReader extends IndexReader {
try {
if (si.hasDeletions()) {
// NOTE: the bitvector is stored using the regular directory, not cfs
liveDocs = new BitVector(directory(), si.getDelFileName(), new IOContext(IOContext.READ, true));
liveDocs = si.getCodec().liveDocsFormat().readLiveDocs(directory(), si, new IOContext(IOContext.READ, true));
} else {
assert si.getDelCount() == 0;
liveDocs = null;
}
numDocs = si.docCount - si.getDelCount();
assert checkLiveCounts(false);
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
@ -76,46 +74,26 @@ public final class SegmentReader extends IndexReader {
}
}
// TODO: really these next 2 ctors could take
// SegmentCoreReaders... that's all we do w/ the parent
// SR:
// Create new SegmentReader sharing core from a previous
// SegmentReader and loading new live docs from a new
// deletes file. Used by openIfChanged.
SegmentReader(SegmentInfo si, SegmentReader parent, IOContext context) throws IOException {
assert si.dir == parent.getSegmentInfo().dir;
this.si = si;
// It's no longer possible to unDeleteAll, so, we can
// only be created if we have deletions:
assert si.hasDeletions();
// ... but load our own deleted docs:
liveDocs = new BitVector(si.dir, si.getDelFileName(), context);
numDocs = si.docCount - si.getDelCount();
assert checkLiveCounts(false);
// We share core w/ parent:
parent.core.incRef();
core = parent.core;
SegmentReader(SegmentInfo si, SegmentCoreReaders core, IOContext context) throws IOException {
this(si, core, si.getCodec().liveDocsFormat().readLiveDocs(si.dir, si, context), si.docCount - si.getDelCount());
}
// Create new SegmentReader sharing core from a previous
// SegmentReader and using the provided in-memory
// liveDocs. Used by IndexWriter to provide a new NRT
// reader:
SegmentReader(SegmentReader parent, BitVector liveDocs, int numDocs) throws IOException {
this.si = parent.si;
parent.core.incRef();
this.core = parent.core;
SegmentReader(SegmentInfo si, SegmentCoreReaders core, Bits liveDocs, int numDocs) throws IOException {
this.si = si;
this.core = core;
core.incRef();
assert liveDocs != null;
this.liveDocs = liveDocs;
this.numDocs = numDocs;
assert checkLiveCounts(true);
}
@Override
@ -124,27 +102,6 @@ public final class SegmentReader extends IndexReader {
return liveDocs;
}
private boolean checkLiveCounts(boolean isNRT) throws IOException {
if (liveDocs != null) {
if (liveDocs.size() != si.docCount) {
throw new CorruptIndexException("document count mismatch: deleted docs count " + liveDocs.size() + " vs segment doc count " + si.docCount + " segment=" + si.name);
}
final int recomputedCount = liveDocs.getRecomputedCount();
// Verify BitVector is self consistent:
assert liveDocs.count() == recomputedCount : "live count=" + liveDocs.count() + " vs recomputed count=" + recomputedCount;
// Verify our docCount matches:
assert numDocs == recomputedCount :
"delete count mismatch: numDocs=" + numDocs + " vs BitVector=" + (si.docCount-recomputedCount);
assert isNRT || si.docCount - si.getDelCount() == recomputedCount :
"si.docCount=" + si.docCount + "si.getDelCount()=" + si.getDelCount() + " recomputedCount=" + recomputedCount;
}
return true;
}
@Override
protected void doClose() throws IOException {
//System.out.println("SR.close seg=" + si);

View File

@ -20,8 +20,8 @@ package org.apache.lucene.index;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.MutableBits;
/**
* @lucene.experimental
@ -32,6 +32,7 @@ public class SegmentWriteState {
public final String segmentName;
public final FieldInfos fieldInfos;
public final int numDocs;
public int delCountOnFlush;
// Deletes to apply while we are flushing the segment. A
// Term is enrolled in here if it was deleted at one
@ -41,7 +42,7 @@ public class SegmentWriteState {
public final BufferedDeletes segDeletes;
// Lazily created:
public BitVector liveDocs;
public MutableBits liveDocs;
public final Codec codec;
public final String segmentSuffix;
@ -83,5 +84,6 @@ public class SegmentWriteState {
codec = state.codec;
this.segmentSuffix = segmentSuffix;
segDeletes = state.segDeletes;
delCountOnFlush = state.delCountOnFlush;
}
}

View File

@ -0,0 +1,22 @@
package org.apache.lucene.util;
/**
* 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.
*/
public interface MutableBits extends Bits {
public void clear(int bit);
}

View File

@ -18,11 +18,15 @@ package org.apache.lucene.codecs.preflexrw;
*/
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.util.LuceneTestCase;
/**
@ -35,6 +39,10 @@ public class PreFlexRWCodec extends Lucene3xCodec {
private final FieldInfosFormat fieldInfos = new PreFlexRWFieldInfosFormat();
private final TermVectorsFormat termVectors = new PreFlexRWTermVectorsFormat();
private final SegmentInfosFormat segmentInfos = new PreFlexRWSegmentInfosFormat();
// TODO: this should really be a different impl
private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
// TODO: this should really be a different impl
private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
@Override
public PostingsFormat postingsFormat() {
@ -80,4 +88,22 @@ public class PreFlexRWCodec extends Lucene3xCodec {
return super.termVectorsFormat();
}
}
@Override
public LiveDocsFormat liveDocsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return liveDocs;
} else {
return super.liveDocsFormat();
}
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
return storedFields;
} else {
return super.storedFieldsFormat();
}
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.util;
package org.apache.lucene.codecs.lucene40;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -22,6 +22,8 @@ import java.io.IOException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
/**
* <code>TestBitVector</code> tests the <code>BitVector</code>, obviously.

View File

@ -27,6 +27,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@ -35,6 +36,7 @@ import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
@ -1156,6 +1158,11 @@ public class TestAddIndexes extends LuceneTestCase {
public NormsFormat normsFormat() {
return new Lucene40NormsFormat();
}
@Override
public LiveDocsFormat liveDocsFormat() {
return new Lucene40LiveDocsFormat();
}
}
/*

View File

@ -497,17 +497,25 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
writer.deleteDocuments(searchTerm);
writer.close();
// Now verify file names:
// Now verify file names... TODO: fix this test better, we could populate from
// separateFiles() or something.
String[] expected = new String[] {"_0.cfs", "_0.cfe",
"_0_1.del",
"segments_2",
"segments.gen"};
String[] expectedSimpleText = new String[] {"_0.cfs", "_0.cfe",
"_0_1.liv",
"segments_2",
"segments.gen"};
String[] actual = dir.listAll();
Arrays.sort(expected);
Arrays.sort(expectedSimpleText);
Arrays.sort(actual);
if (!Arrays.equals(expected, actual)) {
fail("incorrect filenames in index: expected:\n " + asString(expected) + "\n actual:\n " + asString(actual));
if (!Arrays.equals(expected, actual) && !Arrays.equals(expectedSimpleText, actual)) {
fail("incorrect filenames in index: expected:\n " + asString(expected)
+ "\n or " + asString(expectedSimpleText) + "\n actual:\n " + asString(actual));
}
dir.close();
} finally {

View File

@ -87,17 +87,20 @@ public class TestIndexFileDeleter extends LuceneTestCase {
}
*/
// TODO: fix this test better
String ext = Codec.getDefault().getName().equals("SimpleText") ? ".liv" : ".del";
// Create a bogus separate del file for a
// segment that already has a separate del file:
copyFile(dir, "_0_1.del", "_0_2.del");
copyFile(dir, "_0_1" + ext, "_0_2" + ext);
// Create a bogus separate del file for a
// segment that does not yet have a separate del file:
copyFile(dir, "_0_1.del", "_1_1.del");
copyFile(dir, "_0_1" + ext, "_1_1" + ext);
// Create a bogus separate del file for a
// non-existent segment:
copyFile(dir, "_0_1.del", "_188_1.del");
copyFile(dir, "_0_1" + ext, "_188_1" + ext);
// Create a bogus segment file:
copyFile(dir, "_0.cfs", "_188.cfs");

View File

@ -42,7 +42,6 @@ import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;

View File

@ -961,7 +961,8 @@ public class TestIndexWriterDelete extends LuceneTestCase {
}
w.updateDocument(delTerm, doc);
// Eventually segment 0 should get a del docs:
if (dir.fileExists("_0_1.del")) {
// TODO: fix this test
if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv") ) {
if (VERBOSE) {
System.out.println("TEST: deletes created @ count=" + count);
}
@ -1006,7 +1007,8 @@ public class TestIndexWriterDelete extends LuceneTestCase {
}
w.updateDocument(delTerm, doc);
// Eventually segment 0 should get a del docs:
if (dir.fileExists("_0_1.del")) {
// TODO: fix this test
if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
break;
}
count++;
@ -1052,7 +1054,8 @@ public class TestIndexWriterDelete extends LuceneTestCase {
doc.add(newField("body", sb.toString(), TextField.TYPE_UNSTORED));
w.updateDocument(new Term("id", ""+id), doc);
docsInSegment.incrementAndGet();
if (dir.fileExists("_0_1.del")) {
// TODO: fix this test
if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
if (VERBOSE) {
System.out.println("TEST: deletes created @ id=" + id);
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
@ -459,13 +460,13 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
}
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
if ("org.apache.lucene.index.SegmentMerger".equals(trace[i].getClassName()) && "mergeTerms".equals(trace[i].getMethodName()) && !didFail1) {
if (SegmentMerger.class.getName().equals(trace[i].getClassName()) && "mergeTerms".equals(trace[i].getMethodName()) && !didFail1) {
didFail1 = true;
throw new IOException("fake disk full during mergeTerms");
}
if ("org.apache.lucene.util.BitVector".equals(trace[i].getClassName()) && "write".equals(trace[i].getMethodName()) && !didFail2) {
if (LiveDocsFormat.class.getName().equals(trace[i].getClassName()) && "writeLiveDocs".equals(trace[i].getMethodName()) && !didFail2) {
didFail2 = true;
throw new IOException("fake disk full while writing BitVector");
throw new IOException("fake disk full while writing LiveDocs");
}
}
}

View File

@ -0,0 +1,87 @@
package org.apache.lucene.index;
/**
* 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.util.HashSet;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.preflexrw.PreFlexRWCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StringField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
public class TestMixedCodecs extends LuceneTestCase {
public void test() throws Exception {
Assume.assumeTrue(!(Codec.getDefault() instanceof PreFlexRWCodec));
final int NUM_DOCS = atLeast(1000);
final Directory dir = newDirectory();
RandomIndexWriter w = null;
int docsLeftInThisSegment = 0;
int docUpto = 0;
while (docUpto < NUM_DOCS) {
if (docsLeftInThisSegment == 0) {
final IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
if (random.nextBoolean()) {
// Make sure we aggressively mix in SimpleText
// since it has different impls for all codec
// formats...
iwc.setCodec(Codec.forName("SimpleText"));
}
if (w != null) {
w.close();
}
w = new RandomIndexWriter(random, dir, iwc);
docsLeftInThisSegment = _TestUtil.nextInt(random, 10, 100);
}
final Document doc = new Document();
doc.add(newField("id", String.valueOf(docUpto), StringField.TYPE_STORED));
w.addDocument(doc);
docUpto++;
docsLeftInThisSegment--;
}
// Random delete half the docs:
final Set<Integer> deleted = new HashSet<Integer>();
while(deleted.size() < NUM_DOCS/2) {
final Integer toDelete = random.nextInt(NUM_DOCS);
if (!deleted.contains(toDelete)) {
deleted.add(toDelete);
w.deleteDocuments(new Term("id", String.valueOf(toDelete)));
if (random.nextInt(17) == 6) {
final IndexReader r = w.getReader();
assertEquals(NUM_DOCS - deleted.size(), r.numDocs());
r.close();
}
}
}
w.close();
dir.close();
}
}