mirror of https://github.com/apache/lucene.git
LUCENE-8233: Harden PendingDeletes modification assertions
This change adds a missing call to PendingDeletes#onNewReader and hardens the assertion when a PendingDelete can actually be modified ie. receive deletes and updates. Now PendingDeltes are also initialized when no reader is provided but the SegmentCommitInfo has evidence that there is are no deletes.
This commit is contained in:
parent
017f59bae5
commit
9cb07c735b
|
@ -52,7 +52,10 @@ class PendingDeletes {
|
|||
}
|
||||
|
||||
PendingDeletes(SegmentCommitInfo info) {
|
||||
this(info, null, false);
|
||||
this(info, null, info.hasDeletions() == false);
|
||||
// if we don't have deletions we can mark it as initialized since we might receive deletes on a segment
|
||||
// without having a reader opened on it ie. after a merge when we apply the deletes that IW received while merging.
|
||||
// For segments that were published we enforce a reader in the BufferedUpdatesStream.SegmentState ctor
|
||||
}
|
||||
|
||||
private PendingDeletes(SegmentCommitInfo info, Bits liveDocs, boolean liveDocsInitialized) {
|
||||
|
@ -65,6 +68,9 @@ class PendingDeletes {
|
|||
|
||||
|
||||
protected MutableBits getMutableBits() throws IOException {
|
||||
// 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
|
||||
assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
|
||||
if (liveDocsShared) {
|
||||
// Copy on write: this means we've cloned a
|
||||
// SegmentReader sharing the current liveDocs
|
||||
|
|
|
@ -67,7 +67,7 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
|
||||
super.onNewReader(reader, info);
|
||||
hardDeletes.onNewReader(reader, info);
|
||||
if (dvGeneration != info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
|
||||
if (dvGeneration < info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
|
||||
final DocIdSetIterator iterator = DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(field, reader);
|
||||
if (iterator == null) { // nothing is deleted we don't have a soft deletes field in this segment
|
||||
this.pendingDeleteCount = 0;
|
||||
|
@ -120,6 +120,7 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> updatesToApply) throws IOException {
|
||||
if (field.equals(info.name)) {
|
||||
assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
|
||||
assert dvGeneration != -2 : "docValues generation is still uninitialized";
|
||||
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
|
||||
for(int i=0; i<subs.length; i++) {
|
||||
subs[i] = updatesToApply.get(i).iterator();
|
||||
|
|
|
@ -619,6 +619,7 @@ final class ReadersAndUpdates {
|
|||
final SegmentReader reader;
|
||||
if (this.reader == null) {
|
||||
reader = new SegmentReader(info, indexCreatedVersionMajor, IOContext.READONCE);
|
||||
pendingDeletes.onNewReader(reader, info);
|
||||
} else {
|
||||
reader = this.reader;
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertNull(deletes.getLiveDocs());
|
||||
int docToDelete = TestUtil.nextInt(random(), 0, 7);
|
||||
|
@ -76,7 +76,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 6, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertFalse(deletes.writeLiveDocs(dir));
|
||||
assertEquals(0, dir.listAll().length);
|
||||
|
@ -86,7 +86,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
deletes.liveDocsShared();
|
||||
deletes.delete(2);
|
||||
}
|
||||
assertEquals(0, commitInfo.getDelGen());
|
||||
assertEquals(-1, commitInfo.getDelGen());
|
||||
assertEquals(0, commitInfo.getDelCount());
|
||||
|
||||
assertEquals(secondDocDeletes ? 2 : 1, deletes.numPendingDeletes());
|
||||
|
@ -133,7 +133,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
for (int i = 0; i < 3; i++) {
|
||||
assertTrue(deletes.delete(i));
|
||||
|
|
|
@ -45,7 +45,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
|
||||
public void testDeleteSoft() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field hier
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field here
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "1"), doc,
|
||||
|
@ -97,8 +97,20 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig());
|
||||
for (int i = 0; i < si.maxDoc(); i++) {
|
||||
writer.addDocument(new Document());
|
||||
}
|
||||
writer.forceMerge(1);
|
||||
writer.commit();
|
||||
DirectoryReader reader = writer.getReader();
|
||||
assertEquals(1, reader.leaves().size());
|
||||
SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
|
||||
PendingSoftDeletes deletes = newPendingDeletes(commitInfo);
|
||||
deletes.onNewReader(segmentReader, commitInfo);
|
||||
reader.close();
|
||||
writer.close();
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
|
||||
List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
|
||||
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10));
|
||||
|
@ -152,10 +164,9 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertEquals(1, reader.leaves().size());
|
||||
SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
|
||||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
PendingSoftDeletes deletes = newPendingDeletes(segmentInfo);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getDocValuesGen(), Collections.emptyMap(), 0, 0);
|
||||
PendingDeletes deletes = newPendingDeletes(segmentInfo);
|
||||
deletes.onNewReader(segmentReader, segmentInfo);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
|
||||
List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
|
||||
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3));
|
||||
deletes.onDocValuesUpdate(fieldInfo, updates);
|
||||
|
|
Loading…
Reference in New Issue