LUCENE-8303: Make LiveDocsFormat only responsible for serialization/deserialization of live docs.

This commit is contained in:
Adrien Grand 2018-05-09 14:31:23 +02:00
parent b53ee58023
commit 8dc69428e3
16 changed files with 248 additions and 162 deletions

View File

@ -112,6 +112,9 @@ API Changes
* LUCENE-8248: MergePolicyWrapper is renamed to FilterMergePolicy and now * LUCENE-8248: MergePolicyWrapper is renamed to FilterMergePolicy and now
also overrides getMaxCFSSegmentSizeMB (Mike Sokolov via Mike McCandless) also overrides getMaxCFSSegmentSizeMB (Mike Sokolov via Mike McCandless)
* LUCENE-8303: LiveDocsFormat is now only responsible for (de)serialization of
live docs. (Adrien Grand)
New Features New Features
* LUCENE-8200: Allow doc-values to be updated atomically together * LUCENE-8200: Allow doc-values to be updated atomically together

View File

@ -34,7 +34,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder; import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
/** /**
@ -50,17 +49,6 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
final static BytesRef SIZE = new BytesRef("size "); final static BytesRef SIZE = new BytesRef("size ");
final static BytesRef DOC = new BytesRef(" doc "); final static BytesRef DOC = new BytesRef(" doc ");
final static BytesRef END = new BytesRef("END"); 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 @Override
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
@ -107,8 +95,7 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
BitSet set = ((SimpleTextBits) bits).bits;
int size = bits.length(); int size = bits.length();
BytesRefBuilder scratch = new BytesRefBuilder(); BytesRefBuilder scratch = new BytesRefBuilder();
@ -121,10 +108,12 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
SimpleTextUtil.write(out, Integer.toString(size), scratch); SimpleTextUtil.write(out, Integer.toString(size), scratch);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.writeNewline(out);
for (int i = set.nextSetBit(0); i >= 0; i=set.nextSetBit(i + 1)) { for (int i = 0; i < size; ++i) {
SimpleTextUtil.write(out, DOC); if (bits.get(i)) {
SimpleTextUtil.write(out, Integer.toString(i), scratch); SimpleTextUtil.write(out, DOC);
SimpleTextUtil.writeNewline(out); SimpleTextUtil.write(out, Integer.toString(i), scratch);
SimpleTextUtil.writeNewline(out);
}
} }
SimpleTextUtil.write(out, END); SimpleTextUtil.write(out, END);
@ -167,22 +156,5 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
return size; 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

@ -14,18 +14,16 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.lucene.util; package org.apache.lucene.codecs.simpletext;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseLiveDocsFormatTestCase;
/** public class TestSimpleTextLiveDocsFormat extends BaseLiveDocsFormatTestCase {
* Extension of Bits for live documents. private final Codec codec = new SimpleTextCodec();
*/
public interface MutableBits extends Bits { @Override
/** protected Codec getCodec() {
* Sets the bit specified by <code>index</code> to false. return codec;
* @param index index, should be non-negative and &lt; {@link #length()}. }
* The result of passing negative or out of bounds values is undefined
* by this interface, <b>just don't do it!</b>
*/
public void clear(int index);
} }

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
/** Format for live/deleted documents /** Format for live/deleted documents
* @lucene.experimental */ * @lucene.experimental */
@ -35,19 +34,13 @@ public abstract class LiveDocsFormat {
protected LiveDocsFormat() { protected 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;
/** Read live docs bits. */ /** Read live docs bits. */
public abstract Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException; public abstract Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException;
/** Persist live docs bits. Use {@link /** Persist live docs bits. Use {@link
* SegmentCommitInfo#getNextDelGen} to determine the * SegmentCommitInfo#getNextDelGen} to determine the
* generation of the deletes file you should write to. */ * generation of the deletes file you should write to. */
public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException; public abstract void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException;
/** Records all files in use by this {@link SegmentCommitInfo} into the files argument. */ /** Records all files in use by this {@link SegmentCommitInfo} into the files argument. */
public abstract void files(SegmentCommitInfo info, Collection<String> files) throws IOException; public abstract void files(SegmentCommitInfo info, Collection<String> files) throws IOException;

View File

@ -32,7 +32,6 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.MutableBits;
/** /**
* Lucene 5.0 live docs format * Lucene 5.0 live docs format
@ -62,19 +61,6 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
private static final int VERSION_START = 0; private static final int VERSION_START = 0;
private static final int VERSION_CURRENT = VERSION_START; private static final int VERSION_CURRENT = VERSION_START;
@Override
public MutableBits newLiveDocs(int size) throws IOException {
FixedBitSet bits = new FixedBitSet(size);
bits.set(0, size);
return bits;
}
@Override
public MutableBits newLiveDocs(Bits existing) throws IOException {
FixedBitSet fbs = (FixedBitSet) existing;
return fbs.clone();
}
@Override @Override
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
long gen = info.getDelGen(); long gen = info.getDelGen();
@ -105,22 +91,30 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
long gen = info.getNextDelGen(); long gen = info.getNextDelGen();
String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen); String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen);
FixedBitSet fbs = (FixedBitSet) bits; int delCount = 0;
if (fbs.length() - fbs.cardinality() != info.getDelCount() + newDelCount) {
throw new CorruptIndexException("bits.deleted=" + (fbs.length() - fbs.cardinality()) +
" info.delcount=" + info.getDelCount() + " newdelcount=" + newDelCount, name);
}
long data[] = fbs.getBits();
try (IndexOutput output = dir.createOutput(name, context)) { try (IndexOutput output = dir.createOutput(name, context)) {
CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), Long.toString(gen, Character.MAX_RADIX)); CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
for (int i = 0; i < data.length; i++) { final int longCount = FixedBitSet.bits2words(bits.length());
output.writeLong(data[i]); for (int i = 0; i < longCount; ++i) {
long currentBits = 0;
for (int j = i << 6, end = Math.min(j + 63, bits.length() - 1); j <= end; ++j) {
if (bits.get(j)) {
currentBits |= 1L << j; // mod 64
} else {
delCount += 1;
}
}
output.writeLong(currentBits);
} }
CodecUtil.writeFooter(output); CodecUtil.writeFooter(output);
} }
if (delCount != info.getDelCount() + newDelCount) {
throw new CorruptIndexException("bits.deleted=" + delCount +
" info.delcount=" + info.getDelCount() + " newdelcount=" + newDelCount, name);
}
} }
@Override @Override

View File

@ -37,9 +37,9 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.Allocator; import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator; import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.Counter; import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.IntBlockPool; import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
@ -106,12 +106,12 @@ final class DocumentsWriterPerThread {
final SegmentCommitInfo segmentInfo; final SegmentCommitInfo segmentInfo;
final FieldInfos fieldInfos; final FieldInfos fieldInfos;
final FrozenBufferedUpdates segmentUpdates; final FrozenBufferedUpdates segmentUpdates;
final MutableBits liveDocs; final FixedBitSet liveDocs;
final Sorter.DocMap sortMap; final Sorter.DocMap sortMap;
final int delCount; final int delCount;
private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos, private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount, Sorter.DocMap sortMap) BufferedUpdates segmentUpdates, FixedBitSet liveDocs, int delCount, Sorter.DocMap sortMap)
throws IOException { throws IOException {
this.segmentInfo = segmentInfo; this.segmentInfo = segmentInfo;
this.fieldInfos = fieldInfos; this.fieldInfos = fieldInfos;
@ -436,7 +436,8 @@ final class DocumentsWriterPerThread {
// happens when an exception is hit processing that // happens when an exception is hit processing that
// doc, eg if analyzer has some problem w/ the text): // doc, eg if analyzer has some problem w/ the text):
if (pendingUpdates.deleteDocIDs.size() > 0) { if (pendingUpdates.deleteDocIDs.size() > 0) {
flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM); flushState.liveDocs = new FixedBitSet(numDocsInRAM);
flushState.liveDocs.set(0, numDocsInRAM);
for(int delDocID : pendingUpdates.deleteDocIDs) { for(int delDocID : pendingUpdates.deleteDocIDs) {
flushState.liveDocs.clear(delDocID); flushState.liveDocs.clear(delDocID);
} }
@ -529,9 +530,10 @@ final class DocumentsWriterPerThread {
return filesToDelete; return filesToDelete;
} }
private MutableBits sortLiveDocs(Bits liveDocs, Sorter.DocMap sortMap) throws IOException { private FixedBitSet sortLiveDocs(Bits liveDocs, Sorter.DocMap sortMap) throws IOException {
assert liveDocs != null && sortMap != null; assert liveDocs != null && sortMap != null;
MutableBits sortedLiveDocs = codec.liveDocsFormat().newLiveDocs(liveDocs.length()); FixedBitSet sortedLiveDocs = new FixedBitSet(liveDocs.length());
sortedLiveDocs.set(0, liveDocs.length());
for (int i = 0; i < liveDocs.length(); i++) { for (int i = 0; i < liveDocs.length(); i++) {
if (liveDocs.get(i) == false) { if (liveDocs.get(i) == false) {
sortedLiveDocs.clear(sortMap.oldToNew(i)); sortedLiveDocs.clear(sortMap.oldToNew(i));
@ -542,7 +544,7 @@ final class DocumentsWriterPerThread {
/** /**
* Seals the {@link SegmentInfo} for the new flushed segment and persists * Seals the {@link SegmentInfo} for the new flushed segment and persists
* the deleted documents {@link MutableBits}. * the deleted documents {@link FixedBitSet}.
*/ */
void sealFlushedSegment(FlushedSegment flushedSegment, Sorter.DocMap sortMap, DocumentsWriter.FlushNotifications flushNotifications) throws IOException { void sealFlushedSegment(FlushedSegment flushedSegment, Sorter.DocMap sortMap, DocumentsWriter.FlushNotifications flushNotifications) throws IOException {
assert flushedSegment != null; assert flushedSegment != null;
@ -593,7 +595,7 @@ final class DocumentsWriterPerThread {
SegmentCommitInfo info = flushedSegment.segmentInfo; SegmentCommitInfo info = flushedSegment.segmentInfo;
Codec codec = info.info.getCodec(); Codec codec = info.info.getCodec();
final MutableBits bits; final FixedBitSet bits;
if (sortMap == null) { if (sortMap == null) {
bits = flushedSegment.liveDocs; bits = flushedSegment.liveDocs;
} else { } else {

View File

@ -26,6 +26,7 @@ import java.util.Map;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
final class FreqProxTermsWriter extends TermsHash { final class FreqProxTermsWriter extends TermsHash {
@ -63,7 +64,8 @@ final class FreqProxTermsWriter extends TermsHash {
int doc = postingsEnum.nextDoc(); int doc = postingsEnum.nextDoc();
if (doc < delDocLimit) { if (doc < delDocLimit) {
if (state.liveDocs == null) { if (state.liveDocs == null) {
state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.maxDoc()); state.liveDocs = new FixedBitSet(state.segmentInfo.maxDoc());
state.liveDocs.set(0, state.segmentInfo.maxDoc());
} }
if (state.liveDocs.get(doc)) { if (state.liveDocs.get(doc)) {
state.delCountOnFlush++; state.delCountOnFlush++;

View File

@ -20,14 +20,13 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper; import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
/** /**
* This class handles accounting and applying pending deletes for live segment readers * This class handles accounting and applying pending deletes for live segment readers
@ -67,7 +66,7 @@ class PendingDeletes {
} }
protected MutableBits getMutableBits() throws IOException { protected FixedBitSet getMutableBits() throws IOException {
// if we pull mutable bits but we haven't been initialized something is completely off. // if we pull mutable bits but we haven't been initialized something is completely off.
// this means we receive deletes without having the bitset that is on-disk ready to be cloned // this means we receive deletes without having the bitset that is on-disk ready to be cloned
assert liveDocsInitialized : "can't delete if liveDocs are not initialized"; assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
@ -76,17 +75,19 @@ class PendingDeletes {
// SegmentReader sharing the current liveDocs // SegmentReader sharing the current liveDocs
// instance; must now make a private clone so we can // instance; must now make a private clone so we can
// change it: // change it:
LiveDocsFormat liveDocsFormat = info.info.getCodec().liveDocsFormat(); FixedBitSet mutableBits = new FixedBitSet(info.info.maxDoc());
MutableBits mutableBits; mutableBits.set(0, info.info.maxDoc());
if (liveDocs == null) { if (liveDocs != null) {
mutableBits = liveDocsFormat.newLiveDocs(info.info.maxDoc()); for (int i = 0; i < liveDocs.length(); ++i) {
} else { if (liveDocs.get(i) == false) {
mutableBits = liveDocsFormat.newLiveDocs(liveDocs); mutableBits.clear(i);
}
}
} }
liveDocs = mutableBits; liveDocs = mutableBits;
liveDocsShared = false; liveDocsShared = false;
} }
return (MutableBits) liveDocs; return (FixedBitSet) liveDocs;
} }
@ -96,7 +97,7 @@ class PendingDeletes {
*/ */
boolean delete(int docID) throws IOException { boolean delete(int docID) throws IOException {
assert info.info.maxDoc() > 0; assert info.info.maxDoc() > 0;
MutableBits mutableBits = getMutableBits(); FixedBitSet mutableBits = getMutableBits();
assert mutableBits != null; assert mutableBits != null;
assert docID >= 0 && docID < mutableBits.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + mutableBits.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc(); assert docID >= 0 && docID < mutableBits.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + mutableBits.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc();
assert !liveDocsShared; assert !liveDocsShared;
@ -202,7 +203,7 @@ class PendingDeletes {
boolean success = false; boolean success = false;
try { try {
Codec codec = info.info.getCodec(); Codec codec = info.info.getCodec();
codec.liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT); codec.liveDocsFormat().writeLiveDocs(liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
success = true; success = true;
} finally { } finally {
if (!success) { if (!success) {

View File

@ -26,9 +26,9 @@ import org.apache.lucene.search.DocValuesFieldExistsQuery;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
final class PendingSoftDeletes extends PendingDeletes { final class PendingSoftDeletes extends PendingDeletes {
@ -50,7 +50,7 @@ final class PendingSoftDeletes extends PendingDeletes {
@Override @Override
boolean delete(int docID) throws IOException { boolean delete(int docID) throws IOException {
MutableBits mutableBits = getMutableBits(); // we need to fetch this first it might be a shared instance with hardDeletes FixedBitSet mutableBits = getMutableBits(); // we need to fetch this first it might be a shared instance with hardDeletes
if (hardDeletes.delete(docID)) { if (hardDeletes.delete(docID)) {
if (mutableBits.get(docID)) { // delete it here too! if (mutableBits.get(docID)) { // delete it here too!
mutableBits.clear(docID); mutableBits.clear(docID);
@ -105,7 +105,7 @@ final class PendingSoftDeletes extends PendingDeletes {
* @param bits the bit set to apply the deletes to * @param bits the bit set to apply the deletes to
* @return the number of bits changed by this function * @return the number of bits changed by this function
*/ */
static int applySoftDeletes(DocIdSetIterator iterator, MutableBits bits) throws IOException { static int applySoftDeletes(DocIdSetIterator iterator, FixedBitSet bits) throws IOException {
assert iterator != null; assert iterator != null;
int newDeletes = 0; int newDeletes = 0;
int docID; int docID;

View File

@ -21,8 +21,8 @@ import org.apache.lucene.codecs.PostingsFormat; // javadocs
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; // javadocs import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; // javadocs
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.MutableBits;
/** /**
* Holder class for common parameters used during write. * Holder class for common parameters used during write.
@ -56,9 +56,9 @@ public class SegmentWriteState {
*/ */
public final BufferedUpdates segUpdates; public final BufferedUpdates segUpdates;
/** {@link MutableBits} recording live documents; this is /** {@link FixedBitSet} recording live documents; this is
* only set if there is one or more deleted documents. */ * only set if there is one or more deleted documents. */
public MutableBits liveDocs; public FixedBitSet liveDocs;
/** Unique suffix for any postings files written for this /** Unique suffix for any postings files written for this
* segment. {@link PerFieldPostingsFormat} sets this for * segment. {@link PerFieldPostingsFormat} sets this for

View File

@ -25,7 +25,7 @@ import org.apache.lucene.search.DocIdSetIterator;
* Base implementation for a bit set. * Base implementation for a bit set.
* @lucene.internal * @lucene.internal
*/ */
public abstract class BitSet implements MutableBits, Accountable { public abstract class BitSet implements Bits, Accountable {
/** Build a {@link BitSet} from the content of the provided {@link DocIdSetIterator}. /** Build a {@link BitSet} from the content of the provided {@link DocIdSetIterator}.
* NOTE: this will fully consume the {@link DocIdSetIterator}. */ * NOTE: this will fully consume the {@link DocIdSetIterator}. */
@ -45,6 +45,9 @@ public abstract class BitSet implements MutableBits, Accountable {
/** Set the bit at <code>i</code>. */ /** Set the bit at <code>i</code>. */
public abstract void set(int i); public abstract void set(int i);
/** Clear the bit at <code>i</code>. */
public abstract void clear(int i);
/** Clears a range of bits. /** Clears a range of bits.
* *
* @param startIndex lower index * @param startIndex lower index

View File

@ -31,7 +31,7 @@ import org.apache.lucene.search.DocIdSetIterator;
* *
* @lucene.internal * @lucene.internal
*/ */
public final class FixedBitSet extends BitSet implements MutableBits, Accountable { public final class FixedBitSet extends BitSet implements Bits, Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class); private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);

View File

@ -0,0 +1,30 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene50;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseLiveDocsFormatTestCase;
import org.apache.lucene.util.TestUtil;
public class TestLucene50LiveDocsFormat extends BaseLiveDocsFormatTestCase {
@Override
protected Codec getCodec() {
return TestUtil.getDefaultCodec();
}
}

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
/** /**
@ -33,31 +32,6 @@ import org.apache.lucene.util.TestUtil;
public class AssertingLiveDocsFormat extends LiveDocsFormat { public class AssertingLiveDocsFormat extends LiveDocsFormat {
private final LiveDocsFormat in = TestUtil.getDefaultCodec().liveDocsFormat(); private final LiveDocsFormat in = TestUtil.getDefaultCodec().liveDocsFormat();
@Override
public MutableBits newLiveDocs(int size) throws IOException {
assert size >= 0;
MutableBits raw = in.newLiveDocs(size);
assert raw != null;
assert raw.length() == size;
for (int i = 0; i < raw.length(); i++) {
assert raw.get(i);
}
return new AssertingMutableBits(raw);
}
@Override
public MutableBits newLiveDocs(Bits existing) throws IOException {
assert existing instanceof AssertingBits;
Bits rawExisting = ((AssertingBits)existing).in;
MutableBits raw = in.newLiveDocs(rawExisting);
assert raw != null;
assert raw.length() == rawExisting.length();
for (int i = 0; i < raw.length(); i++) {
assert rawExisting.get(i) == raw.get(i);
}
return new AssertingMutableBits(raw);
}
@Override @Override
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
Bits raw = in.readLiveDocs(dir, info, context); Bits raw = in.readLiveDocs(dir, info, context);
@ -67,11 +41,9 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
assert bits instanceof AssertingMutableBits; check(bits, info.info.maxDoc(), info.getDelCount() + newDelCount);
MutableBits raw = (MutableBits) ((AssertingMutableBits)bits).in; in.writeLiveDocs(bits, dir, info, newDelCount, context);
check(raw, info.info.maxDoc(), info.getDelCount() + newDelCount);
in.writeLiveDocs(raw, dir, info, newDelCount, context);
} }
private void check(Bits bits, int expectedLength, int expectedDeleteCount) { private void check(Bits bits, int expectedLength, int expectedDeleteCount) {
@ -120,17 +92,5 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
return "Asserting(" + in + ")"; return "Asserting(" + in + ")";
} }
} }
static class AssertingMutableBits extends AssertingBits implements MutableBits {
AssertingMutableBits(MutableBits in) {
super(in);
}
@Override
public void clear(int index) {
assert index >= 0;
assert index < in.length();
((MutableBits)in).clear(index);
}
}
} }

View File

@ -25,7 +25,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
class CrankyLiveDocsFormat extends LiveDocsFormat { class CrankyLiveDocsFormat extends LiveDocsFormat {
final LiveDocsFormat delegate; final LiveDocsFormat delegate;
@ -36,23 +35,13 @@ class CrankyLiveDocsFormat extends LiveDocsFormat {
this.random = random; this.random = random;
} }
@Override
public MutableBits newLiveDocs(int size) throws IOException {
return delegate.newLiveDocs(size);
}
@Override
public MutableBits newLiveDocs(Bits existing) throws IOException {
return delegate.newLiveDocs(existing);
}
@Override @Override
public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
return delegate.readLiveDocs(dir, info, context); return delegate.readLiveDocs(dir, info, context);
} }
@Override @Override
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
if (random.nextInt(100) == 0) { if (random.nextInt(100) == 0) {
throw new IOException("Fake IOException from LiveDocsFormat.writeLiveDocs()"); throw new IOException("Fake IOException from LiveDocsFormat.writeLiveDocs()");
} }

View File

@ -0,0 +1,139 @@
/*
* 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.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.Version;
/**
* Abstract class that performs basic testing of a codec's
* {@link LiveDocsFormat}.
*/
public abstract class BaseLiveDocsFormatTestCase extends LuceneTestCase {
/** Returns the codec to run tests against */
protected abstract Codec getCodec();
private Codec savedCodec;
@Override
public void setUp() throws Exception {
super.setUp();
// set the default codec, so adding test cases to this isn't fragile
savedCodec = Codec.getDefault();
Codec.setDefault(getCodec());
}
@Override
public void tearDown() throws Exception {
Codec.setDefault(savedCodec); // restore
super.tearDown();
}
public void testDenseLiveDocs() throws IOException {
final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
testSerialization(maxDoc, maxDoc - 1, false);
testSerialization(maxDoc, maxDoc - 1, true);
}
public void testEmptyLiveDocs() throws IOException {
final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
testSerialization(maxDoc, 0, false);
testSerialization(maxDoc, 0, true);
}
public void testSparseLiveDocs() throws IOException {
final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
testSerialization(maxDoc, 1, false);
testSerialization(maxDoc, 1, true);
}
@Nightly
public void testOverflow() throws IOException {
testSerialization(IndexWriter.MAX_DOCS, IndexWriter.MAX_DOCS - 7, false);
}
private void testSerialization(int maxDoc, int numLiveDocs, boolean fixedBitSet) throws IOException {
final Codec codec = Codec.getDefault();
final LiveDocsFormat format = codec.liveDocsFormat();
final FixedBitSet liveDocs = new FixedBitSet(maxDoc);
if (numLiveDocs > maxDoc / 2) {
liveDocs.set(0, maxDoc);
for (int i = 0; i < maxDoc - numLiveDocs; ++i) {
int clearBit;
do {
clearBit = random().nextInt(maxDoc);
} while (liveDocs.get(clearBit) == false);
liveDocs.clear(clearBit);
}
} else {
for (int i = 0; i < numLiveDocs; ++i) {
int setBit;
do {
setBit = random().nextInt(maxDoc);
} while (liveDocs.get(setBit));
liveDocs.set(setBit);
}
}
final Bits bits;
if (fixedBitSet) {
bits = liveDocs;
} else {
// Make sure the impl doesn't only work with a FixedBitSet
bits = new Bits() {
@Override
public boolean get(int index) {
return liveDocs.get(index);
}
@Override
public int length() {
return liveDocs.length();
}
};
}
final Directory dir = newDirectory();
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "foo", maxDoc, random().nextBoolean(),
codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1);
format.writeLiveDocs(bits, dir, sci, maxDoc - numLiveDocs, IOContext.DEFAULT);
sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 1, -1, -1);
final Bits bits2 = format.readLiveDocs(dir, sci, IOContext.READONCE);
assertEquals(maxDoc, bits2.length());
for (int i = 0; i < maxDoc; ++i) {
assertEquals(bits.get(i), bits2.get(i));
}
dir.close();
}
}