mirror of https://github.com/apache/lucene.git
LUCENE-8310: Ensure IndexFileDeleter accounts for pending deletes
Today we fail creating the IndexWriter when the directory has a pending delete. Yet, this is mainly done to prevent writing still existing files more than once. IndexFileDeleter already accounts for that for existing files which we can now use to also take pending deletes into account which ensures that all file generations per segment always go forward.
This commit is contained in:
parent
25ebe0e35f
commit
585952797c
|
@ -182,6 +182,13 @@ New Features
|
|||
TermExclusionConditionalFilter is added as an example. (Alan Woodward,
|
||||
Robert Muir, David Smiley, Steve Rowe, Mike Sokolov)
|
||||
|
||||
* LUCENE-8310: Ensure IndexFileDeleter accounts for pending deletes. Today we fail
|
||||
creating the IndexWriter when the directory has a pending delete. Yet, this
|
||||
is mainly done to prevent writing still existing files more than once.
|
||||
IndexFileDeleter already accounts for that for existing files which we can
|
||||
now use to also take pending deletes into account which ensures that all file
|
||||
generations per segment always go forward. (Simon Willnauer)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-8266: Detect bogus tiles when creating a standard polygon and
|
||||
|
|
|
@ -208,9 +208,13 @@ final class IndexFileDeleter implements Closeable {
|
|||
|
||||
// We keep commits list in sorted order (oldest to newest):
|
||||
CollectionUtil.timSort(commits);
|
||||
|
||||
Collection<String> relevantFiles = new HashSet<>(refCounts.keySet());
|
||||
Set<String> pendingDeletions = directoryOrig.getPendingDeletions();
|
||||
if (pendingDeletions.isEmpty() == false) {
|
||||
relevantFiles.addAll(pendingDeletions);
|
||||
}
|
||||
// refCounts only includes "normal" filenames (does not include write.lock)
|
||||
inflateGens(segmentInfos, refCounts.keySet(), infoStream);
|
||||
inflateGens(segmentInfos, relevantFiles, infoStream);
|
||||
|
||||
// Now delete anything with ref count at 0. These are
|
||||
// presumably abandoned files eg due to crash of
|
||||
|
|
|
@ -695,9 +695,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* IO error
|
||||
*/
|
||||
public IndexWriter(Directory d, IndexWriterConfig conf) throws IOException {
|
||||
if (d.checkPendingDeletions()) {
|
||||
throw new IllegalArgumentException("Directory " + d + " still has pending deleted files; cannot initialize IndexWriter");
|
||||
}
|
||||
enableTestPoints = isEnableTestPoints();
|
||||
conf.setIndexWriter(this); // prevent reuse by other instances
|
||||
config = conf;
|
||||
|
@ -848,6 +845,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
rollbackSegments = segmentInfos.createBackupSegmentInfos();
|
||||
}
|
||||
|
||||
|
||||
|
||||
commitUserData = new HashMap<>(segmentInfos.getUserData()).entrySet();
|
||||
|
||||
pendingNumDocs.set(segmentInfos.totalMaxDoc());
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.util.Collection; // for javadocs
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -172,10 +174,12 @@ public abstract class Directory implements Closeable {
|
|||
*/
|
||||
protected void ensureOpen() throws AlreadyClosedException {}
|
||||
|
||||
/** Tries to delete any pending deleted files, and returns true if
|
||||
* there are still files that could not be deleted.
|
||||
* This method is optional and returns <code>false</code> by default. */
|
||||
public boolean checkPendingDeletions() throws IOException {
|
||||
return false;
|
||||
/**
|
||||
* Returns the current pending deletions in this directory or an emtpy set
|
||||
* if there are no known pending deletions.
|
||||
* @lucene.internal
|
||||
*/
|
||||
public Set<String> getPendingDeletions() throws IOException {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -339,12 +339,6 @@ public abstract class FSDirectory extends BaseDirectory {
|
|||
maybeDeletePendingFiles();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean checkPendingDeletions() throws IOException {
|
||||
deletePendingFiles();
|
||||
return pendingDeletes.isEmpty() == false;
|
||||
}
|
||||
|
||||
/** Try to delete any pending files that we had previously tried to delete but failed
|
||||
* because we are on Windows and the files were still held open. */
|
||||
public synchronized void deletePendingFiles() throws IOException {
|
||||
|
@ -426,4 +420,14 @@ public abstract class FSDirectory extends BaseDirectory {
|
|||
}, CHUNK_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Set<String> getPendingDeletions() throws IOException {
|
||||
deletePendingFiles();
|
||||
if (pendingDeletes.isEmpty()) {
|
||||
return Collections.emptySet();
|
||||
} else {
|
||||
return Collections.unmodifiableSet(new HashSet<>(pendingDeletes));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.nio.file.NoSuchFileException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
@ -204,7 +205,16 @@ public class FileSwitchDirectory extends Directory {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean checkPendingDeletions() throws IOException {
|
||||
return primaryDir.checkPendingDeletions() && secondaryDir.checkPendingDeletions();
|
||||
public Set<String> getPendingDeletions() throws IOException {
|
||||
Set<String> primaryDeletions = primaryDir.getPendingDeletions();
|
||||
Set<String> secondaryDeletions = secondaryDir.getPendingDeletions();
|
||||
if (primaryDeletions.isEmpty() && secondaryDeletions.isEmpty()) {
|
||||
return Collections.emptySet();
|
||||
} else {
|
||||
HashSet<String> combined = new HashSet<>();
|
||||
combined.addAll(primaryDeletions);
|
||||
combined.addAll(secondaryDeletions);
|
||||
return Collections.unmodifiableSet(combined);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.store;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
/** Directory implementation that delegates calls to another directory.
|
||||
* This class can be used to add limitations on top of an existing
|
||||
|
@ -115,7 +116,7 @@ public abstract class FilterDirectory extends Directory {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean checkPendingDeletions() throws IOException {
|
||||
return in.checkPendingDeletions();
|
||||
public Set<String> getPendingDeletions() throws IOException {
|
||||
return super.getPendingDeletions();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.index;
|
|||
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
@ -40,6 +41,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.CannedTokenStream;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
|
@ -2641,7 +2643,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IOUtils.close(r, r2, w, dir);
|
||||
}
|
||||
|
||||
public void testWithPendingDeletions() throws Exception {
|
||||
public void testPendingDeleteDVGeneration() throws IOException {
|
||||
// irony: currently we don't emulate windows well enough to work on windows!
|
||||
assumeFalse("windows is not supported", Constants.WINDOWS);
|
||||
|
||||
|
@ -2651,6 +2653,115 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
FileSystem fs = new WindowsFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
|
||||
Path root = new FilterPath(path, fs);
|
||||
|
||||
// MMapDirectory doesn't work because it closes its file handles after mapping!
|
||||
List<Closeable> toClose = new ArrayList<>();
|
||||
try (FSDirectory dir = new SimpleFSDirectory(root);
|
||||
Closeable closeable = () -> IOUtils.close(toClose)) {
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setUseCompoundFile(false)
|
||||
.setMaxBufferedDocs(2)
|
||||
.setRAMBufferSizeMB(-1);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
d.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(d);
|
||||
d = new Document();
|
||||
d.add(new StringField("id", "2", Field.Store.YES));
|
||||
d.add(new NumericDocValuesField("id", 2));
|
||||
w.addDocument(d);
|
||||
w.flush();
|
||||
d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
d.add(new NumericDocValuesField("id", 1));
|
||||
w.updateDocument(new Term("id", "1"), d);
|
||||
w.commit();
|
||||
Set<String> files = new HashSet<>(Arrays.asList(dir.listAll()));
|
||||
int numIters = 10 + random().nextInt(50);
|
||||
for (int i = 0; i < numIters; i++) {
|
||||
if (random().nextBoolean()) {
|
||||
d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
d.add(new NumericDocValuesField("id", 1));
|
||||
w.updateDocument(new Term("id", "1"), d);
|
||||
} else if (random().nextBoolean()) {
|
||||
w.deleteDocuments(new Term("id", "2"));
|
||||
} else {
|
||||
w.updateNumericDocValue(new Term("id", "1"), "id", 2);
|
||||
}
|
||||
w.prepareCommit();
|
||||
List<String> newFiles = new ArrayList<>(Arrays.asList(dir.listAll()));
|
||||
newFiles.removeAll(files);
|
||||
String randomFile = RandomPicks.randomFrom(random(), newFiles);
|
||||
toClose.add(dir.openInput(randomFile, IOContext.DEFAULT));
|
||||
w.rollback();
|
||||
iwc = new IndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setUseCompoundFile(false)
|
||||
.setMaxBufferedDocs(2)
|
||||
.setRAMBufferSizeMB(-1);
|
||||
w = new IndexWriter(dir, iwc);
|
||||
expectThrows(NoSuchFileException.class, () -> {
|
||||
dir.deleteFile(randomFile);
|
||||
});
|
||||
}
|
||||
w.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testWithPendingDeletions() throws Exception {
|
||||
// irony: currently we don't emulate windows well enough to work on windows!
|
||||
assumeFalse("windows is not supported", Constants.WINDOWS);
|
||||
|
||||
Path path = createTempDir();
|
||||
|
||||
// Use WindowsFS to prevent open files from being deleted:
|
||||
FileSystem fs = new WindowsFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
|
||||
Path root = new FilterPath(path, fs);
|
||||
IndexCommit indexCommit;
|
||||
DirectoryReader reader;
|
||||
// MMapDirectory doesn't work because it closes its file handles after mapping!
|
||||
try (FSDirectory dir = new SimpleFSDirectory(root)) {
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random())).setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
w.commit();
|
||||
reader = w.getReader();
|
||||
// we pull this commit to open it again later to check that we fail if a futur file delete is pending
|
||||
indexCommit = reader.getIndexCommit();
|
||||
w.close();
|
||||
w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE));
|
||||
w.addDocument(new Document());
|
||||
w.close();
|
||||
IndexInput in = dir.openInput("segments_2", IOContext.DEFAULT);
|
||||
dir.deleteFile("segments_2");
|
||||
assertTrue(dir.getPendingDeletions().size() > 0);
|
||||
|
||||
// make sure we get NFSF if we try to delete and already-pending-delete file:
|
||||
expectThrows(NoSuchFileException.class, () -> {
|
||||
dir.deleteFile("segments_2");
|
||||
});
|
||||
|
||||
try (IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setIndexCommit(indexCommit))) {
|
||||
writer.addDocument(new Document());
|
||||
writer.commit();
|
||||
assertEquals(1, writer.maxDoc());
|
||||
// now check that we moved to 3
|
||||
dir.openInput("segments_3", IOContext.READ).close();;
|
||||
}
|
||||
reader.close();
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testPendingDeletesAlreadyWrittenFiles() throws IOException {
|
||||
Path path = createTempDir();
|
||||
// irony: currently we don't emulate windows well enough to work on windows!
|
||||
assumeFalse("windows is not supported", Constants.WINDOWS);
|
||||
|
||||
// Use WindowsFS to prevent open files from being deleted:
|
||||
FileSystem fs = new WindowsFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
|
||||
Path root = new FilterPath(path, fs);
|
||||
DirectoryReader reader;
|
||||
// MMapDirectory doesn't work because it closes its file handles after mapping!
|
||||
try (FSDirectory dir = new SimpleFSDirectory(root)) {
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -2659,18 +2770,14 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexInput in = dir.openInput("segments_1", IOContext.DEFAULT);
|
||||
w.addDocument(new Document());
|
||||
w.close();
|
||||
assertTrue(dir.checkPendingDeletions());
|
||||
|
||||
assertTrue(dir.getPendingDeletions().size() > 0);
|
||||
|
||||
// make sure we get NFSF if we try to delete and already-pending-delete file:
|
||||
expectThrows(NoSuchFileException.class, () -> {
|
||||
dir.deleteFile("segments_1");
|
||||
});
|
||||
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("still has pending deleted files; cannot initialize IndexWriter"));
|
||||
|
||||
new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random()))).close();
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -66,10 +65,6 @@ public class TestIndexWriterOutOfFileDescriptors extends LuceneTestCase {
|
|||
if (ms instanceof ConcurrentMergeScheduler) {
|
||||
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||
}
|
||||
if (Constants.WINDOWS && dir.checkPendingDeletions()) {
|
||||
// if we are on windows and we have pending deletions we can't execute this test
|
||||
break;
|
||||
}
|
||||
w = new IndexWriter(dir, iwc);
|
||||
if (r != null && random().nextInt(5) == 3) {
|
||||
if (random().nextBoolean()) {
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.lucene.index.SegmentInfos;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
@ -211,14 +210,10 @@ public class TestDirectoryTaxonomyWriter extends FacetTestCase {
|
|||
// now recreate the taxonomy, and check that the epoch is preserved after opening DirTW again.
|
||||
taxoWriter.close();
|
||||
|
||||
assumeFalse("if we are on windows and we have pending deletions we can't execute this test",
|
||||
Constants.WINDOWS && dir.checkPendingDeletions());
|
||||
taxoWriter = new DirectoryTaxonomyWriter(dir, OpenMode.CREATE, NO_OP_CACHE);
|
||||
touchTaxo(taxoWriter, new FacetLabel("c"));
|
||||
taxoWriter.close();
|
||||
|
||||
assumeFalse("if we are on windows and we have pending deletions we can't execute this test",
|
||||
Constants.WINDOWS && dir.checkPendingDeletions());
|
||||
taxoWriter = new DirectoryTaxonomyWriter(dir, OpenMode.CREATE_OR_APPEND, NO_OP_CACHE);
|
||||
touchTaxo(taxoWriter, new FacetLabel("d"));
|
||||
taxoWriter.close();
|
||||
|
|
|
@ -82,7 +82,7 @@ public abstract class ReplicaNode extends Node {
|
|||
public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory, PrintStream printStream) throws IOException {
|
||||
super(id, dir, searcherFactory, printStream);
|
||||
|
||||
if (dir.checkPendingDeletions()) {
|
||||
if (dir.getPendingDeletions().isEmpty() == false) {
|
||||
throw new IllegalArgumentException("Directory " + dir + " still has pending deleted files; cannot initialize IndexWriter");
|
||||
}
|
||||
|
||||
|
@ -199,7 +199,7 @@ public abstract class ReplicaNode extends Node {
|
|||
assert deleter.getRefCount(segmentsFileName) == 1;
|
||||
deleter.decRef(Collections.singleton(segmentsFileName));
|
||||
|
||||
if (dir.checkPendingDeletions()) {
|
||||
if (dir.getPendingDeletions().isEmpty() == false) {
|
||||
// If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else there is a definite window during
|
||||
// which if we carsh, we cause corruption:
|
||||
throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed");
|
||||
|
|
|
@ -1149,7 +1149,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
|||
out.getFilePointer(); // just fake access to prevent compiler warning
|
||||
}
|
||||
fsDir.deleteFile(candidate);
|
||||
if (fsDir.checkPendingDeletions()) {
|
||||
if (fsDir.getPendingDeletions().size() > 0) {
|
||||
// good: virus checker struck and prevented deletion of fileName
|
||||
fileName = candidate;
|
||||
break;
|
||||
|
|
|
@ -231,11 +231,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
|
|||
printStream.println("\nTEST: WriterThread iter=" + i);
|
||||
iwc.setOpenMode(OpenMode.APPEND);
|
||||
try {
|
||||
if (Constants.WINDOWS && dir.checkPendingDeletions()) {
|
||||
// on windows we can potentially have pending deletes here if this happens we step out like in the catch clause
|
||||
// tests using this also assumes no mock window FS
|
||||
break;
|
||||
}
|
||||
writer = new IndexWriter(dir, iwc);
|
||||
|
||||
} catch (Throwable t) {
|
||||
|
|
Loading…
Reference in New Issue