LUCENE-3439: catch more cases where IW/IR is used after close; fix sneaky bug where IW can delete files after it was closed
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1173720 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index cd1cb47..42bb29a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -616,6 +616,10 @@
directly into the FieldCache arrays during sorting, leading to concurrency
issues. (Uwe Schindler, Doron Cohen, Mike McCandless)
+* LUCENE-3439: Closing an NRT reader after the writer was closed was
+ incorrectly invoking the DeletionPolicy and (then possibly deleting
+ files) on the closed IndexWriter (Robert Muir, Mike McCandless)
+
New Features
Optimizations
diff --git a/lucene/src/java/org/apache/lucene/index/DirectoryReader.java b/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
index 3618ac5..ad46bdb 100644
--- a/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
+++ b/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
@@ -364,6 +364,7 @@
@Override
public final synchronized IndexReader clone(boolean openReadOnly) throws CorruptIndexException, IOException {
+ // doReopen calls ensureOpen
DirectoryReader newReader = doReopen((SegmentInfos) segmentInfos.clone(), true, openReadOnly);
if (this != newReader) {
@@ -566,7 +567,7 @@
@Override
public boolean hasDeletions() {
- // Don't call ensureOpen() here (it could affect performance)
+ ensureOpen();
return hasDeletions;
}
@@ -843,7 +844,7 @@
if (writer != null) {
// Since we just closed, writer may now be able to
// delete unused files:
- writer.deleteUnusedFiles();
+ writer.deletePendingFiles();
}
// throw the first exception
@@ -868,6 +869,7 @@
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return topLevelReaderContext;
}
@@ -887,6 +889,7 @@
@Override
public int getTermInfosIndexDivisor() {
+ ensureOpen();
return termInfosIndexDivisor;
}
@@ -897,6 +900,7 @@
*/
@Override
public IndexCommit getIndexCommit() throws IOException {
+ ensureOpen();
return new ReaderCommit(segmentInfos, directory);
}
diff --git a/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java b/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
index 7fecceb..22b6788 100644
--- a/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
+++ b/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
@@ -306,11 +306,13 @@
@Override
public Directory directory() {
+ ensureOpen();
return in.directory();
}
@Override
public Bits getLiveDocs() {
+ ensureOpen();
return in.getLiveDocs();
}
@@ -333,7 +335,6 @@
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
ensureOpen();
in.getTermFreqVector(docNumber, field, mapper);
-
}
@Override
@@ -362,7 +363,7 @@
@Override
public boolean hasDeletions() {
- // Don't call ensureOpen() here (it could affect performance)
+ ensureOpen();
return in.hasDeletions();
}
@@ -402,14 +403,15 @@
protected void doDelete(int n) throws CorruptIndexException, IOException { in.deleteDocument(n); }
@Override
- protected void doCommit(Map<String,String> commitUserData) throws IOException { in.commit(commitUserData); }
+ protected void doCommit(Map<String,String> commitUserData) throws IOException {
+ in.commit(commitUserData);
+ }
@Override
protected void doClose() throws IOException {
in.close();
}
-
@Override
public Collection<String> getFieldNames(IndexReader.FieldOption fieldNames) {
ensureOpen();
@@ -441,11 +443,13 @@
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return in.getTopReaderContext();
}
@Override
public Fields fields() throws IOException {
+ ensureOpen();
return in.fields();
}
@@ -480,6 +484,7 @@
@Override
public PerDocValues perDocValues() throws IOException {
+ ensureOpen();
return in.perDocValues();
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/IndexReader.java b/lucene/src/java/org/apache/lucene/index/IndexReader.java
index e94b3af..8604c64 100644
--- a/lucene/src/java/org/apache/lucene/index/IndexReader.java
+++ b/lucene/src/java/org/apache/lucene/index/IndexReader.java
@@ -115,6 +115,7 @@
*
* @lucene.experimental */
public void addReaderFinishedListener(ReaderFinishedListener listener) {
+ ensureOpen();
readerFinishedListeners.add(listener);
}
@@ -122,6 +123,7 @@
*
* @lucene.experimental */
public void removeReaderFinishedListener(ReaderFinishedListener listener) {
+ ensureOpen();
readerFinishedListeners.remove(listener);
}
@@ -172,7 +174,7 @@
DOC_VALUES
}
- private boolean closed;
+ private volatile boolean closed;
protected boolean hasChanges;
private final AtomicInteger refCount = new AtomicInteger();
@@ -236,7 +238,8 @@
*/
public void decRef() throws IOException {
ensureOpen();
- if (refCount.getAndDecrement() == 1) {
+ final int rc = refCount.getAndDecrement();
+ if (rc == 1) {
boolean success = false;
try {
commit();
@@ -249,6 +252,8 @@
}
}
readerFinished();
+ } else if (rc <= 0) {
+ throw new IllegalStateException("too many decRef calls: refCount was " + rc + " before decrement");
}
}
@@ -1330,6 +1335,7 @@
* @throws IOException if there is a low-level IO error
*/
public final synchronized void commit(Map<String, String> commitUserData) throws IOException {
+ // Don't can ensureOpen since we commit() on close
doCommit(commitUserData);
hasChanges = false;
}
@@ -1499,6 +1505,7 @@
* corruption for other readers (like DirectoryReader obtained
* through {@link #open}. Use the parent reader directly. */
public IndexReader[] getSequentialSubReaders() {
+ ensureOpen();
return null;
}
@@ -1531,6 +1538,8 @@
/** Expert */
public Object getCoreCacheKey() {
+ // Don't can ensureOpen since FC calls this (to evict)
+ // on close
return this;
}
@@ -1569,6 +1578,7 @@
}
public final IndexDocValues docValues(String field) throws IOException {
+ ensureOpen();
final PerDocValues perDoc = perDocValues();
if (perDoc == null) {
return null;
@@ -1580,11 +1590,13 @@
/** @lucene.internal */
void storeFields(Fields fields) {
+ ensureOpen();
this.fields = fields;
}
/** @lucene.internal */
Fields retrieveFields() {
+ ensureOpen();
return fields;
}
@@ -1592,11 +1604,13 @@
/** @lucene.internal */
void storePerDoc(PerDocValues perDocValues) {
+ ensureOpen();
this.perDocValues = perDocValues;
}
/** @lucene.internal */
PerDocValues retrievePerDoc() {
+ ensureOpen();
return perDocValues;
}
diff --git a/lucene/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/src/java/org/apache/lucene/index/IndexWriter.java
index 87be6b5..f3fa1dc 100644
--- a/lucene/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/src/java/org/apache/lucene/index/IndexWriter.java
@@ -233,8 +233,8 @@
private Lock writeLock;
- private boolean closed;
- private boolean closing;
+ private volatile boolean closed;
+ private volatile boolean closing;
// Holds all SegmentInfo instances currently involved in
// merges
@@ -758,6 +758,7 @@
* delCount is returned.
*/
public int numDeletedDocs(SegmentInfo info) throws IOException {
+ ensureOpen(false);
SegmentReader reader = readerPool.getIfExists(info);
try {
if (reader != null) {
@@ -960,6 +961,7 @@
* IndexWriterConfig} for details.
*/
public IndexWriterConfig getConfig() {
+ ensureOpen(false);
return config;
}
@@ -1204,6 +1206,7 @@
* not counting deletions.
* @see #numDocs */
public synchronized int maxDoc() {
+ ensureOpen();
int count;
if (docWriter != null)
count = docWriter.getNumDocs();
@@ -1221,6 +1224,7 @@
* counted you should call {@link #commit()} first.
* @see #numDocs */
public synchronized int numDocs() throws IOException {
+ ensureOpen();
int count;
if (docWriter != null)
count = docWriter.getNumDocs();
@@ -1960,6 +1964,7 @@
}
private final void maybeMerge(int maxNumSegmentsOptimize, boolean optimize) throws CorruptIndexException, IOException {
+ ensureOpen(false);
updatePendingMerges(maxNumSegmentsOptimize, optimize);
mergeScheduler.merge(this);
}
@@ -2143,6 +2148,7 @@
* {@link MergePolicy.MergeAbortedException}s.
*/
public synchronized void deleteAll() throws IOException {
+ ensureOpen();
try {
// Abort any running merges
@@ -2229,6 +2235,7 @@
* will have completed once this method completes.</p>
*/
public synchronized void waitForMerges() {
+ ensureOpen(false);
if (infoStream != null) {
message("waitForMerges");
}
@@ -2736,6 +2743,7 @@
* index to commit.
*/
public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
+ ensureOpen(false);
if (infoStream != null) {
message("prepareCommit: flush");
@@ -4098,10 +4106,16 @@
* be deleted the next time commit() is called.
*/
public synchronized void deleteUnusedFiles() throws IOException {
+ ensureOpen(false);
deleter.deletePendingFiles();
deleter.revisitPolicy();
}
+ // Called by DirectoryReader.doClose
+ synchronized void deletePendingFiles() throws IOException {
+ deleter.deletePendingFiles();
+ }
+
/**
* Sets the {@link PayloadProcessorProvider} to use when merging payloads.
* Note that the given <code>pcp</code> will be invoked for every segment that
@@ -4121,6 +4135,7 @@
* {@link #optimize}, you can call {@link #waitForMerges()} before.
*/
public void setPayloadProcessorProvider(PayloadProcessorProvider pcp) {
+ ensureOpen();
payloadProcessorProvider = pcp;
}
@@ -4129,6 +4144,7 @@
* merges to process payloads.
*/
public PayloadProcessorProvider getPayloadProcessorProvider() {
+ ensureOpen();
return payloadProcessorProvider;
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/MultiReader.java b/lucene/src/java/org/apache/lucene/index/MultiReader.java
index 72062cf..dfdc0db 100644
--- a/lucene/src/java/org/apache/lucene/index/MultiReader.java
+++ b/lucene/src/java/org/apache/lucene/index/MultiReader.java
@@ -233,6 +233,7 @@
@Override
public boolean isOptimized() {
+ ensureOpen();
return false;
}
@@ -265,7 +266,7 @@
@Override
public boolean hasDeletions() {
- // Don't call ensureOpen() here (it could affect performance)
+ ensureOpen();
return hasDeletions;
}
@@ -358,6 +359,7 @@
*/
@Override
public boolean isCurrent() throws CorruptIndexException, IOException {
+ ensureOpen();
for (int i = 0; i < subReaders.length; i++) {
if (!subReaders[i].isCurrent()) {
return false;
@@ -383,6 +385,7 @@
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return topLevelContext;
}
diff --git a/lucene/src/java/org/apache/lucene/index/ParallelReader.java b/lucene/src/java/org/apache/lucene/index/ParallelReader.java
index 22c006c..df4b731 100644
--- a/lucene/src/java/org/apache/lucene/index/ParallelReader.java
+++ b/lucene/src/java/org/apache/lucene/index/ParallelReader.java
@@ -204,16 +204,19 @@
@Override
public Bits getLiveDocs() {
+ ensureOpen();
return MultiFields.getLiveDocs(readers.get(0));
}
@Override
public Fields fields() {
+ ensureOpen();
return fields;
}
@Override
public synchronized Object clone() {
+ // doReopen calls ensureOpen
try {
return doReopen(true);
} catch (Exception ex) {
@@ -242,6 +245,7 @@
*/
@Override
public synchronized IndexReader reopen() throws CorruptIndexException, IOException {
+ // doReopen calls ensureOpen
return doReopen(false);
}
@@ -325,7 +329,7 @@
@Override
public boolean hasDeletions() {
- // Don't call ensureOpen() here (it could affect performance)
+ ensureOpen();
return hasDeletions;
}
@@ -461,6 +465,7 @@
*/
@Override
public boolean isCurrent() throws CorruptIndexException, IOException {
+ ensureOpen();
for (final IndexReader reader : readers) {
if (!reader.isCurrent()) {
return false;
@@ -476,6 +481,7 @@
*/
@Override
public boolean isOptimized() {
+ ensureOpen();
for (final IndexReader reader : readers) {
if (!reader.isOptimized()) {
return false;
@@ -527,8 +533,10 @@
}
return fieldSet;
}
+
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return topLevelReaderContext;
}
@@ -550,6 +558,7 @@
@Override
public PerDocValues perDocValues() throws IOException {
+ ensureOpen();
return perDocs;
}
diff --git a/lucene/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/src/java/org/apache/lucene/index/SegmentReader.java
index 622e45e..10f1cf9 100644
--- a/lucene/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/src/java/org/apache/lucene/index/SegmentReader.java
@@ -135,6 +135,7 @@
@Override
public Bits getLiveDocs() {
+ ensureOpen();
return liveDocs;
}
@@ -184,6 +185,7 @@
* @return New BitVector
*/
protected BitVector cloneDeletedDocs(BitVector bv) {
+ ensureOpen();
return (BitVector)bv.clone();
}
@@ -214,6 +216,7 @@
}
synchronized SegmentReader reopenSegment(SegmentInfo si, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
+ ensureOpen();
boolean deletionsUpToDate = (this.si.hasDeletions() == si.hasDeletions())
&& (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
boolean normsUpToDate = true;
@@ -463,13 +466,13 @@
@Override
public Fields fields() throws IOException {
+ ensureOpen();
return core.fields;
}
@Override
public int docFreq(String field, BytesRef term) throws IOException {
ensureOpen();
-
Terms terms = core.fields.terms(field);
if (terms != null) {
return terms.docFreq(term);
@@ -758,6 +761,7 @@
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return readerContext;
}
@@ -835,6 +839,7 @@
@Override
public PerDocValues perDocValues() throws IOException {
+ ensureOpen();
return core.perDocProducer;
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java b/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
index e9432bd..39e15a6 100644
--- a/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
+++ b/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
@@ -67,16 +67,19 @@
@Override
public Fields fields() throws IOException {
+ ensureOpen();
return MultiFields.getFields(in);
}
@Override
public PerDocValues perDocValues() throws IOException {
+ ensureOpen();
return MultiPerDocValues.getPerDocs(in);
}
@Override
public Bits getLiveDocs() {
+ ensureOpen();
return MultiFields.getLiveDocs(in);
}
@@ -103,6 +106,7 @@
@Override
public ReaderContext getTopReaderContext() {
+ ensureOpen();
return readerContext;
}
diff --git a/lucene/src/test/org/apache/lucene/index/TestIndexReader.java b/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
index 1e3269c..c71e6cb 100644
--- a/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
+++ b/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
@@ -73,18 +73,16 @@
IndexReader r = IndexReader.open(d, false);
r.deleteDocument(5);
r.flush(commitUserData);
+ IndexCommit c = r.getIndexCommit();
r.close();
SegmentInfos sis = new SegmentInfos();
sis.read(d);
IndexReader r2 = IndexReader.open(d, false);
- IndexCommit c = r.getIndexCommit();
assertEquals(c.getUserData(), commitUserData);
assertEquals(sis.getCurrentSegmentFileName(), c.getSegmentsFileName());
- assertTrue(c.equals(r.getIndexCommit()));
-
// Change the index
writer = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random)).setOpenMode(
diff --git a/lucene/src/test/org/apache/lucene/index/TestReaderClosed.java b/lucene/src/test/org/apache/lucene/index/TestReaderClosed.java
new file mode 100644
index 0000000..a870123
--- /dev/null
+++ b/lucene/src/test/org/apache/lucene/index/TestReaderClosed.java
@@ -0,0 +1,79 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermRangeQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+public class TestReaderClosed extends LuceneTestCase {
+ private IndexSearcher searcher;
+ private IndexReader reader;
+ private Directory dir;
+
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ dir = newDirectory();
+ RandomIndexWriter writer = new RandomIndexWriter(random, dir,
+ newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.KEYWORD, false))
+ .setMaxBufferedDocs(_TestUtil.nextInt(random, 50, 1000)));
+
+ Document doc = new Document();
+ Field field = newField("field", "", StringField.TYPE_UNSTORED);
+ doc.add(field);
+
+ // we generate aweful prefixes: good for testing.
+ // but for preflex codec, the test can be very slow, so use less iterations.
+ final String codec = CodecProvider.getDefault().getFieldCodec("field");
+ int num = codec.equals("PreFlex") ? 200 * RANDOM_MULTIPLIER : atLeast(1000);
+ for (int i = 0; i < num; i++) {
+ field.setValue(_TestUtil.randomUnicodeString(random, 10));
+ writer.addDocument(doc);
+ }
+ reader = writer.getReader();
+ searcher = newSearcher(reader);
+ writer.close();
+ }
+
+ public void test() throws Exception {
+ TermRangeQuery query = TermRangeQuery.newStringRange("field", "a", "z", true, true);
+ searcher.search(query, 5);
+ searcher.close();
+ reader.close();
+ try {
+ searcher.search(query, 5);
+ } catch (AlreadyClosedException ace) {
+ // expected
+ }
+ }
+
+ public void tearDown() throws Exception {
+ dir.close();
+ super.tearDown();
+ }
+}