mirror of https://github.com/apache/lucene.git
LUCENE-8735: Avoid FileAlreadyExistsException on windows. (#619)
FilterDirectory.getPendingDeletions() did not delegate the call, which resulted in a new IndexWriter on same directory not considering pending delete files. This could in turn result in a FileAlreadyExistsException when running windows.
This commit is contained in:
parent
65165c7c87
commit
04afdb6442
|
@ -7,6 +7,10 @@ http://s.apache.org/luceneversions
|
||||||
|
|
||||||
API Changes
|
API Changes
|
||||||
|
|
||||||
|
* LUCENE-8735: Directory.getPendingDeletions is now abstract to ensure
|
||||||
|
subclasses override it. FilterDirectory now delegates the call, ensuring
|
||||||
|
correct default behaviour for subclasses. (Henning Andersen)
|
||||||
|
|
||||||
* LUCENE-8474: RAMDirectory and associated deprecated classes have been
|
* LUCENE-8474: RAMDirectory and associated deprecated classes have been
|
||||||
removed. (Dawid Weiss)
|
removed. (Dawid Weiss)
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,9 @@ import java.text.DecimalFormatSymbols;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.codecs.CompoundFormat;
|
import org.apache.lucene.codecs.CompoundFormat;
|
||||||
import org.apache.lucene.index.CorruptIndexException;
|
import org.apache.lucene.index.CorruptIndexException;
|
||||||
|
@ -136,6 +138,11 @@ public class SimpleTextCompoundFormat extends CompoundFormat {
|
||||||
in.close();
|
in.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getPendingDeletions() throws IOException {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
|
||||||
// write methods: disabled
|
// write methods: disabled
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.codecs.CodecUtil;
|
import org.apache.lucene.codecs.CodecUtil;
|
||||||
import org.apache.lucene.index.CorruptIndexException;
|
import org.apache.lucene.index.CorruptIndexException;
|
||||||
|
@ -212,4 +213,9 @@ final class Lucene50CompoundReader extends Directory {
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "CompoundFileDirectory(segment=\"" + segmentName + "\" in dir=" + directory + ")";
|
return "CompoundFileDirectory(segment=\"" + segmentName + "\" in dir=" + directory + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getPendingDeletions() {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,9 +23,11 @@ import java.nio.file.FileAlreadyExistsException;
|
||||||
import java.nio.file.NoSuchFileException;
|
import java.nio.file.NoSuchFileException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.function.BiFunction;
|
import java.util.function.BiFunction;
|
||||||
|
@ -229,6 +231,11 @@ public final class ByteBuffersDirectory extends BaseDirectory {
|
||||||
files.clear();
|
files.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getPendingDeletions() {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
|
||||||
private final class FileEntry {
|
private final class FileEntry {
|
||||||
private final String fileName;
|
private final String fileName;
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.NoSuchFileException;
|
import java.nio.file.NoSuchFileException;
|
||||||
import java.util.Collection; // for javadocs
|
import java.util.Collection; // for javadocs
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
|
@ -207,7 +206,5 @@ public abstract class Directory implements Closeable {
|
||||||
*
|
*
|
||||||
* @lucene.internal
|
* @lucene.internal
|
||||||
*/
|
*/
|
||||||
public Set<String> getPendingDeletions() throws IOException {
|
public abstract Set<String> getPendingDeletions() throws IOException;
|
||||||
return Collections.emptySet();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -117,6 +117,6 @@ public abstract class FilterDirectory extends Directory {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<String> getPendingDeletions() throws IOException {
|
public Set<String> getPendingDeletions() throws IOException {
|
||||||
return super.getPendingDeletions();
|
return in.getPendingDeletions();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -2737,6 +2737,45 @@ public class TestIndexWriter extends LuceneTestCase {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testPendingDeletionsRollbackWithReader() throws IOException {
|
||||||
|
// 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);
|
||||||
|
try (FSDirectory _dir = new SimpleFSDirectory(root)) {
|
||||||
|
Directory dir = new FilterDirectory(_dir) {};
|
||||||
|
|
||||||
|
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||||
|
IndexWriter w = new IndexWriter(dir, iwc);
|
||||||
|
Document d = new Document();
|
||||||
|
d.add(new StringField("id", "1", Field.Store.YES));
|
||||||
|
d.add(new NumericDocValuesField("numval", 1));
|
||||||
|
w.addDocument(d);
|
||||||
|
w.commit();
|
||||||
|
w.addDocument(d);
|
||||||
|
w.flush();
|
||||||
|
DirectoryReader reader = DirectoryReader.open(w);
|
||||||
|
w.rollback();
|
||||||
|
|
||||||
|
// try-delete superfluous files (some will fail due to windows-fs)
|
||||||
|
IndexWriterConfig iwc2 = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||||
|
new IndexWriter(dir, iwc2).close();
|
||||||
|
|
||||||
|
// test that we can index on top of pending deletions
|
||||||
|
IndexWriterConfig iwc3 = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||||
|
w = new IndexWriter(dir, iwc3);
|
||||||
|
w.addDocument(d);
|
||||||
|
w.commit();
|
||||||
|
|
||||||
|
reader.close();
|
||||||
|
w.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void testWithPendingDeletions() throws Exception {
|
public void testWithPendingDeletions() throws Exception {
|
||||||
// irony: currently we don't emulate windows well enough to work on windows!
|
// irony: currently we don't emulate windows well enough to work on windows!
|
||||||
assumeFalse("windows is not supported", Constants.WINDOWS);
|
assumeFalse("windows is not supported", Constants.WINDOWS);
|
||||||
|
|
|
@ -277,6 +277,10 @@ public abstract class BaseMergePolicyTestCase extends LuceneTestCase {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getPendingDeletions() throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -21,7 +21,9 @@ import java.lang.invoke.MethodHandles;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
@ -196,6 +198,11 @@ public class HdfsDirectory extends BaseDirectory {
|
||||||
return configuration;
|
return configuration;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getPendingDeletions() {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
|
||||||
public static class HdfsIndexInput extends CustomBufferedIndexInput {
|
public static class HdfsIndexInput extends CustomBufferedIndexInput {
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue