LUCENE-8324: Checkpoint after fully deletes segment is dropped on flush

This commit is contained in:
Simon Willnauer 2018-05-20 14:06:14 +02:00
parent 669b9e7a53
commit 2ce53791d3
2 changed files with 37 additions and 0 deletions

View File

@ -2650,6 +2650,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
try {
if (isFullyDeleted(rld)) {
dropDeletedSegment(newSegment);
checkpoint();
}
} finally {
release(rld);

View File

@ -40,6 +40,7 @@ import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.analysis.Analyzer;
@ -3314,4 +3315,39 @@ public class TestIndexWriter extends LuceneTestCase {
t.join();
}
}
private static void assertFiles(IndexWriter writer) throws IOException {
// remove segment files we don't know if we have committed and what is kept around
Set<String> segFiles = new HashSet<>(writer.segmentInfos.files(true)).stream()
.filter(f -> f.startsWith("segments") == false).collect(Collectors.toSet());
Set<String> dirFiles = new HashSet<>(Arrays.asList(writer.getDirectory().listAll()))
.stream().filter(f -> f.startsWith("segments") == false).collect(Collectors.toSet());
Set<String> s = new HashSet<>(segFiles);
s.removeAll(dirFiles);
assertEquals(segFiles.toString() + " vs "+ dirFiles.toString(), segFiles.size(), dirFiles.size());
}
public void testFullyDeletedSegmentsReleaseFiles() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig config = newIndexWriterConfig();
config.setRAMBufferSizeMB(Integer.MAX_VALUE);
config.setMaxBufferedDocs(2); // no auto flush
IndexWriter writer = new IndexWriter(dir, config);
Document d = new Document();
d.add(new StringField("id", "doc-0", Field.Store.YES));
writer.addDocument(d);
writer.flush();
d = new Document();
d.add(new StringField("id", "doc-1", Field.Store.YES));
writer.addDocument(d);
writer.deleteDocuments(new Term("id", "doc-1"));
assertEquals(1, writer.segmentInfos.asList().size());
writer.flush();
assertEquals(1, writer.segmentInfos.asList().size());
writer.commit();
assertFiles(writer);
assertEquals(1, writer.segmentInfos.asList().size());
IOUtils.close(writer, dir);
}
}