You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/12/06 00:06:20 UTC
svn commit: r1643466 - in /lucene/dev/branches/branch_5x: ./ lucene/
lucene/core/ lucene/core/src/java/org/apache/lucene/index/
lucene/core/src/test/org/apache/lucene/codecs/compressing/
lucene/core/src/test/org/apache/lucene/index/ lucene/sandbox/ luc...
Author: mikemccand
Date: Fri Dec 5 23:06:19 2014
New Revision: 1643466
URL: http://svn.apache.org/r1643466
Log:
LUCENE-5987: IndexWriter forcefully closes itself on hitting aborting exception
Added:
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/AbortingException.java
- copied unchanged from r1643432, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AbortingException.java
lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/SuppressingConcurrentMergeScheduler.java
- copied unchanged from r1643432, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/SuppressingConcurrentMergeScheduler.java
Removed:
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterAbort.java
Modified:
lucene/dev/branches/branch_5x/ (props changed)
lucene/dev/branches/branch_5x/lucene/ (props changed)
lucene/dev/branches/branch_5x/lucene/CHANGES.txt (contents, props changed)
lucene/dev/branches/branch_5x/lucene/core/ (props changed)
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java (contents, props changed)
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java
lucene/dev/branches/branch_5x/lucene/sandbox/ (props changed)
lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java
lucene/dev/branches/branch_5x/lucene/test-framework/ (props changed)
Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Fri Dec 5 23:06:19 2014
@@ -337,6 +337,10 @@ Bug Fixes
* LUCENE-6075: Don't overflow int in SimpleRateLimiter (Boaz Leskes
via Mike McCandless)
+
+* LUCENE-5987: IndexWriter will now forcefully close itself on
+ aborting exception (an exception that would otherwise cause silent
+ data loss). (Robert Muir, Mike McCandless)
Documentation
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Fri Dec 5 23:06:19 2014
@@ -83,7 +83,7 @@ final class DefaultIndexingChain extends
}
@Override
- public void flush(SegmentWriteState state) throws IOException {
+ public void flush(SegmentWriteState state) throws IOException, AbortingException {
// NOTE: caller (DocumentsWriterPerThread) handles
// aborting on any exception from this method
@@ -176,7 +176,7 @@ final class DefaultIndexingChain extends
/** Catch up for all docs before us that had no stored
* fields, or hit non-aborting exceptions before writing
* stored fields. */
- private void fillStoredFields(int docID) throws IOException {
+ private void fillStoredFields(int docID) throws IOException, AbortingException {
while (lastStoredDocID < docID) {
startStoredFields();
finishStoredFields();
@@ -253,36 +253,28 @@ final class DefaultIndexingChain extends
/** Calls StoredFieldsWriter.startDocument, aborting the
* segment if it hits any exception. */
- private void startStoredFields() throws IOException {
- boolean success = false;
+ private void startStoredFields() throws IOException, AbortingException {
try {
initStoredFieldsWriter();
storedFieldsWriter.startDocument();
- success = true;
- } finally {
- if (success == false) {
- docWriter.setAborting();
- }
+ } catch (Throwable th) {
+ throw new AbortingException(th);
}
lastStoredDocID++;
}
/** Calls StoredFieldsWriter.finishDocument, aborting the
* segment if it hits any exception. */
- private void finishStoredFields() throws IOException {
- boolean success = false;
+ private void finishStoredFields() throws IOException, AbortingException {
try {
storedFieldsWriter.finishDocument();
- success = true;
- } finally {
- if (success == false) {
- docWriter.setAborting();
- }
+ } catch (Throwable th) {
+ throw new AbortingException(th);
}
}
@Override
- public void processDocument() throws IOException {
+ public void processDocument() throws IOException, AbortingException {
// How many indexed field names we've seen (collapses
// multiple field instances by the same name):
@@ -302,12 +294,16 @@ final class DefaultIndexingChain extends
fillStoredFields(docState.docID);
startStoredFields();
+ boolean aborting = false;
try {
for (IndexableField field : docState.doc) {
fieldCount = processField(field, fieldGen, fieldCount);
}
+ } catch (AbortingException ae) {
+ aborting = true;
+ throw ae;
} finally {
- if (docWriter.aborting == false) {
+ if (aborting == false) {
// Finish each indexed field name seen in the document:
for (int i=0;i<fieldCount;i++) {
fields[i].finish();
@@ -316,20 +312,16 @@ final class DefaultIndexingChain extends
}
}
- boolean success = false;
try {
termsHash.finishDocument();
- success = true;
- } finally {
- if (success == false) {
- // Must abort, on the possibility that on-disk term
- // vectors are now corrupt:
- docWriter.setAborting();
- }
+ } catch (Throwable th) {
+ // Must abort, on the possibility that on-disk term
+ // vectors are now corrupt:
+ throw new AbortingException(th);
}
}
- private int processField(IndexableField field, long fieldGen, int fieldCount) throws IOException {
+ private int processField(IndexableField field, long fieldGen, int fieldCount) throws IOException, AbortingException {
String fieldName = field.name();
IndexableFieldType fieldType = field.fieldType();
@@ -369,10 +361,8 @@ final class DefaultIndexingChain extends
try {
storedFieldsWriter.writeField(fp.fieldInfo, field);
success = true;
- } finally {
- if (!success) {
- docWriter.setAborting();
- }
+ } catch (Throwable th) {
+ throw new AbortingException(th);
}
}
}
@@ -587,7 +577,7 @@ final class DefaultIndexingChain extends
/** Inverts one field for one document; first is true
* if this is the first time we are seeing this field
* name in this document. */
- public void invert(IndexableField field, boolean first) throws IOException {
+ public void invert(IndexableField field, boolean first) throws IOException, AbortingException {
if (first) {
// First time we're seeing this field (indexed) in
// this document:
@@ -614,7 +604,6 @@ final class DefaultIndexingChain extends
* when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
* but rather a finally that takes note of the problem.
*/
- boolean aborting = false;
boolean succeededInProcessingField = false;
try (TokenStream stream = tokenStream = field.tokenStream(docState.analyzer, tokenStream)) {
// reset the TokenStream to the first token
@@ -666,9 +655,21 @@ final class DefaultIndexingChain extends
// internal state of the terms hash is now
// corrupt and should not be flushed to a
// new segment:
- aborting = true;
- termsHashPerField.add();
- aborting = false;
+ try {
+ termsHashPerField.add();
+ } catch (MaxBytesLengthExceededException e) {
+ byte[] prefix = new byte[30];
+ BytesRef bigTerm = invertState.termAttribute.getBytesRef();
+ System.arraycopy(bigTerm.bytes, bigTerm.offset, prefix, 0, 30);
+ String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + Arrays.toString(prefix) + "...', original message: " + e.getMessage();
+ if (docState.infoStream.isEnabled("IW")) {
+ docState.infoStream.message("IW", "ERROR: " + msg);
+ }
+ // Document will be deleted above:
+ throw new IllegalArgumentException(msg, e);
+ } catch (Throwable th) {
+ throw new AbortingException(th);
+ }
}
// trigger streams to perform end-of-stream operations
@@ -681,22 +682,7 @@ final class DefaultIndexingChain extends
/* if there is an exception coming through, we won't set this to true here:*/
succeededInProcessingField = true;
- } catch (MaxBytesLengthExceededException e) {
- aborting = false;
- byte[] prefix = new byte[30];
- BytesRef bigTerm = invertState.termAttribute.getBytesRef();
- System.arraycopy(bigTerm.bytes, bigTerm.offset, prefix, 0, 30);
- String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + Arrays.toString(prefix) + "...', original message: " + e.getMessage();
- if (docState.infoStream.isEnabled("IW")) {
- docState.infoStream.message("IW", "ERROR: " + msg);
- }
- // Document will be deleted above:
- throw new IllegalArgumentException(msg, e);
} finally {
- if (succeededInProcessingField == false && aborting) {
- docState.docWriter.setAborting();
- }
-
if (!succeededInProcessingField && docState.infoStream.isEnabled("DW")) {
docState.infoStream.message("DW", "An exception was thrown while processing field " + fieldInfo.name);
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Fri Dec 5 23:06:19 2014
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
abstract class DocConsumer {
- abstract void processDocument() throws IOException;
- abstract void flush(final SegmentWriteState state) throws IOException;
+ abstract void processDocument() throws IOException, AbortingException;
+ abstract void flush(final SegmentWriteState state) throws IOException, AbortingException;
abstract void abort();
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Dec 5 23:06:19 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
/**
@@ -204,7 +205,6 @@ final class DocumentsWriter implements C
synchronized void abort(IndexWriter writer) {
assert !Thread.holdsLock(writer) : "IndexWriter lock should never be hold when aborting";
boolean success = false;
- final Set<String> newFilesSet = new HashSet<>();
try {
deleteQueue.clear();
if (infoStream.isEnabled("DW")) {
@@ -215,18 +215,17 @@ final class DocumentsWriter implements C
final ThreadState perThread = perThreadPool.getThreadState(i);
perThread.lock();
try {
- abortThreadState(perThread, newFilesSet);
+ abortThreadState(perThread);
} finally {
perThread.unlock();
}
}
- flushControl.abortPendingFlushes(newFilesSet);
- putEvent(new DeleteNewFilesEvent(newFilesSet));
+ flushControl.abortPendingFlushes();
flushControl.waitForFlush();
success = true;
} finally {
if (infoStream.isEnabled("DW")) {
- infoStream.message("DW", "done abort; abortedFiles=" + newFilesSet + " success=" + success);
+ infoStream.message("DW", "done abort success=" + success);
}
}
}
@@ -240,15 +239,13 @@ final class DocumentsWriter implements C
try {
deleteQueue.clear();
final int limit = perThreadPool.getMaxThreadStates();
- final Set<String> newFilesSet = new HashSet<>();
for (int i = 0; i < limit; i++) {
final ThreadState perThread = perThreadPool.getThreadState(i);
perThread.lock();
- abortThreadState(perThread, newFilesSet);
+ abortThreadState(perThread);
}
deleteQueue.clear();
- flushControl.abortPendingFlushes(newFilesSet);
- putEvent(new DeleteNewFilesEvent(newFilesSet));
+ flushControl.abortPendingFlushes();
flushControl.waitForFlush();
success = true;
} finally {
@@ -262,15 +259,14 @@ final class DocumentsWriter implements C
}
}
- private final void abortThreadState(final ThreadState perThread, Set<String> newFiles) {
+ private final void abortThreadState(final ThreadState perThread) {
assert perThread.isHeldByCurrentThread();
if (perThread.isActive()) { // we might be closed
if (perThread.isInitialized()) {
try {
subtractFlushedNumDocs(perThread.dwpt.getNumDocsInRAM());
- perThread.dwpt.abort(newFiles);
+ perThread.dwpt.abort();
} finally {
- perThread.dwpt.checkAndResetHasAborted();
flushControl.doOnAbort(perThread);
}
} else {
@@ -341,7 +337,7 @@ final class DocumentsWriter implements C
flushControl.setClosed();
}
- private boolean preUpdate() throws IOException {
+ private boolean preUpdate() throws IOException, AbortingException {
ensureOpen();
boolean hasEvents = false;
if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
@@ -373,7 +369,7 @@ final class DocumentsWriter implements C
return hasEvents;
}
- private boolean postUpdate(DocumentsWriterPerThread flushingDWPT, boolean hasEvents) throws IOException {
+ private boolean postUpdate(DocumentsWriterPerThread flushingDWPT, boolean hasEvents) throws IOException, AbortingException {
hasEvents |= applyAllDeletes(deleteQueue);
if (flushingDWPT != null) {
hasEvents |= doFlush(flushingDWPT);
@@ -398,7 +394,7 @@ final class DocumentsWriter implements C
}
boolean updateDocuments(final Iterable<? extends Iterable<? extends IndexableField>> docs, final Analyzer analyzer,
- final Term delTerm) throws IOException {
+ final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
final ThreadState perThread = flushControl.obtainAndLock();
@@ -415,18 +411,15 @@ final class DocumentsWriter implements C
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocuments(docs, analyzer, delTerm);
+ } catch (AbortingException ae) {
+ flushControl.doOnAbort(perThread);
+ dwpt.abort();
+ throw ae;
} finally {
// We don't know how many documents were actually
// counted as indexed, so we must subtract here to
// accumulate our separate counter:
numDocsInRAM.addAndGet(dwpt.getNumDocsInRAM() - dwptNumDocs);
- if (dwpt.checkAndResetHasAborted()) {
- if (!dwpt.pendingFilesToDelete().isEmpty()) {
- putEvent(new DeleteNewFilesEvent(dwpt.pendingFilesToDelete()));
- }
- subtractFlushedNumDocs(dwptNumDocs);
- flushControl.doOnAbort(perThread);
- }
}
final boolean isUpdate = delTerm != null;
flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate);
@@ -438,7 +431,7 @@ final class DocumentsWriter implements C
}
boolean updateDocument(final Iterable<? extends IndexableField> doc, final Analyzer analyzer,
- final Term delTerm) throws IOException {
+ final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
@@ -456,18 +449,15 @@ final class DocumentsWriter implements C
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocument(doc, analyzer, delTerm);
+ } catch (AbortingException ae) {
+ flushControl.doOnAbort(perThread);
+ dwpt.abort();
+ throw ae;
} finally {
// We don't know whether the document actually
// counted as being indexed, so we must subtract here to
// accumulate our separate counter:
numDocsInRAM.addAndGet(dwpt.getNumDocsInRAM() - dwptNumDocs);
- if (dwpt.checkAndResetHasAborted()) {
- if (!dwpt.pendingFilesToDelete().isEmpty()) {
- putEvent(new DeleteNewFilesEvent(dwpt.pendingFilesToDelete()));
- }
- subtractFlushedNumDocs(dwptNumDocs);
- flushControl.doOnAbort(perThread);
- }
}
final boolean isUpdate = delTerm != null;
flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate);
@@ -478,12 +468,13 @@ final class DocumentsWriter implements C
return postUpdate(flushingDWPT, hasEvents);
}
- private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException {
+ private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException, AbortingException {
boolean hasEvents = false;
while (flushingDWPT != null) {
hasEvents = true;
boolean success = false;
SegmentFlushTicket ticket = null;
+ Throwable exc = null;
try {
assert currentFullFlushDelQueue == null
|| flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: "
@@ -549,7 +540,6 @@ final class DocumentsWriter implements C
}
} finally {
flushControl.doAfterFlush(flushingDWPT);
- flushingDWPT.checkAndResetHasAborted();
}
flushingDWPT = flushControl.nextPendingFlush();
@@ -598,8 +588,8 @@ final class DocumentsWriter implements C
* two stage operation; the caller must ensure (in try/finally) that finishFlush
* is called after this method, to release the flush lock in DWFlushControl
*/
- final boolean flushAllThreads(final IndexWriter indexWriter)
- throws IOException {
+ final boolean flushAllThreads()
+ throws IOException, AbortingException {
final DocumentsWriterDeleteQueue flushingDeleteQueue;
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "startFullFlush");
@@ -632,7 +622,7 @@ final class DocumentsWriter implements C
}
ticketQueue.addDeletes(flushingDeleteQueue);
}
- ticketQueue.forcePurge(indexWriter);
+ ticketQueue.forcePurge(writer);
assert !flushingDeleteQueue.anyChanges() && !ticketQueue.hasTickets();
} finally {
assert flushingDeleteQueue == currentFullFlushDelQueue;
@@ -650,9 +640,7 @@ final class DocumentsWriter implements C
// Release the flush lock
flushControl.finishFullFlush();
} else {
- Set<String> newFilesSet = new HashSet<>();
- flushControl.abortFullFlushes(newFilesSet);
- putEvent(new DeleteNewFilesEvent(newFilesSet));
+ flushControl.abortFullFlushes();
}
} finally {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Fri Dec 5 23:06:19 2014
@@ -311,7 +311,7 @@ final class DocumentsWriterFlushControl
dwpt = perThreadPool.reset(perThread, closed);
numPending--;
blockedFlushes.add(new BlockedFlush(dwpt, bytes));
- }finally {
+ } finally {
perThread.unlock();
}
}
@@ -617,20 +617,20 @@ final class DocumentsWriterFlushControl
return true;
}
- synchronized void abortFullFlushes(Set<String> newFiles) {
+ synchronized void abortFullFlushes() {
try {
- abortPendingFlushes(newFiles);
+ abortPendingFlushes();
} finally {
fullFlush = false;
}
}
- synchronized void abortPendingFlushes(Set<String> newFiles) {
+ synchronized void abortPendingFlushes() {
try {
for (DocumentsWriterPerThread dwpt : flushQueue) {
try {
documentsWriter.subtractFlushedNumDocs(dwpt.getNumDocsInRAM());
- dwpt.abort(newFiles);
+ dwpt.abort();
} catch (Throwable ex) {
// ignore - keep on aborting the flush queue
} finally {
@@ -642,7 +642,7 @@ final class DocumentsWriterFlushControl
flushingWriters
.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.getNumDocsInRAM());
- blockedFlush.dwpt.abort(newFiles);
+ blockedFlush.dwpt.abort();
} catch (Throwable ex) {
// ignore - keep on aborting the blocked queue
} finally {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Dec 5 23:06:19 2014
@@ -112,9 +112,9 @@ class DocumentsWriterPerThread {
* updating the index files) and must discard all
* currently buffered docs. This resets our state,
* discarding any docs added since last flush. */
- void abort(Set<String> createdFiles) {
+ void abort() {
//System.out.println(Thread.currentThread().getName() + ": now abort seg=" + segmentInfo.name);
- hasAborted = aborting = true;
+ aborted = true;
try {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "now abort");
@@ -125,9 +125,7 @@ class DocumentsWriterPerThread {
}
pendingUpdates.clear();
- createdFiles.addAll(directory.getCreatedFiles());
} finally {
- aborting = false;
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "done abort");
}
@@ -145,8 +143,7 @@ class DocumentsWriterPerThread {
// Updates for our still-in-RAM (to be flushed next) segment
final BufferedUpdates pendingUpdates;
private final SegmentInfo segmentInfo; // Current segment we are working on
- boolean aborting = false; // True if an abort is pending
- boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting
+ boolean aborted = false; // True if we aborted
private final FieldInfos.Builder fieldInfos;
private final InfoStream infoStream;
@@ -189,20 +186,10 @@ class DocumentsWriterPerThread {
consumer = indexWriterConfig.getIndexingChain().getChain(this);
}
- void setAborting() {
- aborting = true;
- }
-
public FieldInfos.Builder getFieldInfosBuilder() {
return fieldInfos;
}
- boolean checkAndResetHasAborted() {
- final boolean retval = hasAborted;
- hasAborted = false;
- return retval;
- }
-
final void testPoint(String message) {
if (infoStream.isEnabled("TP")) {
infoStream.message("TP", message);
@@ -219,7 +206,7 @@ class DocumentsWriterPerThread {
}
}
- public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException {
+ public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null;
docState.doc = doc;
@@ -245,19 +232,15 @@ class DocumentsWriterPerThread {
success = true;
} finally {
if (!success) {
- if (!aborting) {
- // mark document as deleted
- deleteDocID(docState.docID);
- numDocsInRAM++;
- } else {
- abort(filesToDelete);
- }
+ // mark document as deleted
+ deleteDocID(docState.docID);
+ numDocsInRAM++;
}
}
finishDocument(delTerm);
}
- public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException {
+ public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocuments start");
assert deleteQueue != null;
docState.analyzer = analyzer;
@@ -285,14 +268,9 @@ class DocumentsWriterPerThread {
success = true;
} finally {
if (!success) {
- // An exc is being thrown...
- if (!aborting) {
- // Incr here because finishDocument will not
- // be called (because an exc is being thrown):
- numDocsInRAM++;
- } else {
- abort(filesToDelete);
- }
+ // Incr here because finishDocument will not
+ // be called (because an exc is being thrown):
+ numDocsInRAM++;
}
}
finishDocument(null);
@@ -309,7 +287,7 @@ class DocumentsWriterPerThread {
}
} finally {
- if (!allDocsIndexed && !aborting) {
+ if (!allDocsIndexed && !aborted) {
// the iterator threw an exception that is not aborting
// go and mark all docs from this block as deleted
int docID = numDocsInRAM-1;
@@ -402,7 +380,7 @@ class DocumentsWriterPerThread {
}
/** Flush all pending docs to a new segment */
- FlushedSegment flush() throws IOException {
+ FlushedSegment flush() throws IOException, AbortingException {
assert numDocsInRAM > 0;
assert deleteSlice.isEmpty() : "all deletes must be applied in prepareFlush";
segmentInfo.setDocCount(numDocsInRAM);
@@ -423,7 +401,7 @@ class DocumentsWriterPerThread {
pendingUpdates.docIDs.clear();
}
- if (aborting) {
+ if (aborted) {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "flush: skip because aborting is set");
}
@@ -434,8 +412,6 @@ class DocumentsWriterPerThread {
infoStream.message("DWPT", "flush postings as segment " + flushState.segmentInfo.name + " numDocs=" + numDocsInRAM);
}
- boolean success = false;
-
try {
consumer.flush(flushState);
pendingUpdates.terms.clear();
@@ -475,13 +451,11 @@ class DocumentsWriterPerThread {
FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
sealFlushedSegment(fs);
- success = true;
return fs;
- } finally {
- if (!success) {
- abort(filesToDelete);
- }
+ } catch (Throwable th) {
+ abort();
+ throw new AbortingException(th);
}
}
@@ -602,7 +576,7 @@ class DocumentsWriterPerThread {
@Override
public String toString() {
return "DocumentsWriterPerThread [pendingDeletes=" + pendingUpdates
- + ", segment=" + (segmentInfo != null ? segmentInfo.name : "null") + ", aborting=" + aborting + ", numDocsInRAM="
+ + ", segment=" + (segmentInfo != null ? segmentInfo.name : "null") + ", aborted=" + aborted + ", numDocsInRAM="
+ numDocsInRAM + ", deleteQueue=" + deleteQueue + "]";
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Fri Dec 5 23:06:19 2014
@@ -343,7 +343,7 @@ final class IndexFileDeleter implements
}
}
- private void ensureOpen() throws AlreadyClosedException {
+ void ensureOpen() throws AlreadyClosedException {
writer.ensureOpen(false);
// since we allow 'closing' state, we must still check this, we could be closing because we hit e.g. OOM
if (writer.tragedy != null) {
@@ -351,6 +351,16 @@ final class IndexFileDeleter implements
}
}
+ // for testing
+ boolean isClosed() {
+ try {
+ ensureOpen();
+ return false;
+ } catch (AlreadyClosedException ace) {
+ return true;
+ }
+ }
+
public SegmentInfos getLastSegmentInfos() {
return lastSegmentInfos;
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Dec 5 23:06:19 2014
@@ -405,7 +405,7 @@ public class IndexWriter implements Clos
synchronized (fullFlushLock) {
boolean success = false;
try {
- anySegmentFlushed = docWriter.flushAllThreads(this);
+ anySegmentFlushed = docWriter.flushAllThreads();
if (!anySegmentFlushed) {
// prevent double increment since docWriter#doFlush increments the flushcount
// if we flushed anything.
@@ -422,8 +422,8 @@ public class IndexWriter implements Clos
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "getReader");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "getReader");
// never reached but javac disagrees:
return null;
} finally {
@@ -1236,8 +1236,8 @@ public class IndexWriter implements Clos
}
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "updateDocuments");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "updateDocuments");
}
}
@@ -1406,8 +1406,8 @@ public class IndexWriter implements Clos
}
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "updateDocument");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "updateDocument");
}
}
@@ -2728,7 +2728,7 @@ public class IndexWriter implements Clos
boolean flushSuccess = false;
boolean success = false;
try {
- anySegmentsFlushed = docWriter.flushAllThreads(this);
+ anySegmentsFlushed = docWriter.flushAllThreads();
if (!anySegmentsFlushed) {
// prevent double increment since docWriter#doFlush increments the flushcount
// if we flushed anything.
@@ -2771,8 +2771,8 @@ public class IndexWriter implements Clos
doAfterFlush();
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "prepareCommit");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "prepareCommit");
}
boolean success = false;
@@ -3013,7 +3013,7 @@ public class IndexWriter implements Clos
synchronized (fullFlushLock) {
boolean flushSuccess = false;
try {
- anySegmentFlushed = docWriter.flushAllThreads(this);
+ anySegmentFlushed = docWriter.flushAllThreads();
flushSuccess = true;
} finally {
docWriter.finishFullFlush(flushSuccess);
@@ -3030,8 +3030,8 @@ public class IndexWriter implements Clos
success = true;
return anySegmentFlushed;
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "doFlush");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "doFlush");
// never hit
return false;
} finally {
@@ -4386,19 +4386,28 @@ public class IndexWriter implements Clos
public abstract void warm(LeafReader reader) throws IOException;
}
- private void tragicEvent(Throwable tragedy, String location) {
+ void tragicEvent(Throwable tragedy, String location) throws IOException {
+ // unbox our internal AbortingException
+ if (tragedy instanceof AbortingException) {
+ tragedy = tragedy.getCause();
+ }
// We cannot hold IW's lock here else it can lead to deadlock:
assert Thread.holdsLock(this) == false;
+ // How can it be a tragedy when nothing happened?
+ assert tragedy != null;
+
if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
+ infoStream.message("IW", "hit tragic " + tragedy.getClass().getSimpleName() + " inside " + location);
}
+
synchronized (this) {
- // its possible you could have a really bad day
+ // it's possible you could have a really bad day
if (this.tragedy == null) {
this.tragedy = tragedy;
}
}
+
// if we are already closed (e.g. called by rollback), this will be a no-op.
synchronized(commitLock) {
if (closing == false) {
@@ -4410,7 +4419,8 @@ public class IndexWriter implements Clos
}
}
}
- IOUtils.reThrowUnchecked(tragedy);
+
+ IOUtils.reThrow(tragedy);
}
// Used for testing. Current points:
@@ -4576,11 +4586,13 @@ public class IndexWriter implements Clos
}
private boolean processEvents(Queue<Event> queue, boolean triggerMerge, boolean forcePurge) throws IOException {
- Event event;
boolean processed = false;
- while((event = queue.poll()) != null) {
- processed = true;
- event.process(this, triggerMerge, forcePurge);
+ if (tragedy == null) {
+ Event event;
+ while((event = queue.poll()) != null) {
+ processed = true;
+ event.process(this, triggerMerge, forcePurge);
+ }
}
return processed;
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Fri Dec 5 23:06:19 2014
@@ -142,6 +142,9 @@ public final class IndexWriterConfig ext
* if this config is already attached to a writer.
*/
IndexWriterConfig setIndexWriter(IndexWriter writer) {
+ if (this.writer.get() != null) {
+ throw new IllegalStateException("do not share IndexWriterConfig instances across IndexWriters");
+ }
this.writer.set(writer);
return this;
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Fri Dec 5 23:06:19 2014
@@ -22,17 +22,17 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.IntField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.junit.Test;
-
import com.carrotsearch.randomizedtesting.generators.RandomInts;
public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
@@ -84,17 +84,7 @@ public class TestCompressingStoredFields
iw.commit();
}
finally {
- // next event will cause IW to delete the old files: we use prepareCommit just as example
- iw.prepareCommit();
- int counter = 0;
- for (String fileName : dir.listAll()) {
- if (fileName.endsWith(".fdt") || fileName.endsWith(".fdx")) {
- counter++;
- }
- }
- // Only one .fdt and one .fdx files must have been found
- assertEquals(2, counter);
- iw.rollback();
+ // Abort should have closed the deleter:
dir.close();
}
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Fri Dec 5 23:06:19 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.LuceneTestCase;
@@ -81,14 +82,23 @@ public class TestConcurrentMergeSchedule
MockDirectoryWrapper directory = newMockDirectory();
FailOnlyOnFlush failure = new FailOnlyOnFlush();
directory.failOn(failure);
-
- IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random()))
- .setMaxBufferedDocs(2));
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
+ .setMaxBufferedDocs(2);
+ if (iwc.getMergeScheduler() instanceof ConcurrentMergeScheduler) {
+ iwc.setMergeScheduler(new SuppressingConcurrentMergeScheduler() {
+ @Override
+ protected boolean isOK(Throwable th) {
+ return th instanceof AlreadyClosedException ||
+ (th instanceof IllegalStateException && th.getMessage().contains("this writer hit an unrecoverable error"));
+ }
+ });
+ }
+ IndexWriter writer = new IndexWriter(directory, iwc);
Document doc = new Document();
Field idField = newStringField("id", "", Field.Store.YES);
doc.add(idField);
- int extraCount = 0;
+ outer:
for(int i=0;i<10;i++) {
if (VERBOSE) {
System.out.println("TEST: iter=" + i);
@@ -110,22 +120,20 @@ public class TestConcurrentMergeSchedule
if (failure.hitExc) {
fail("failed to hit IOException");
}
- extraCount++;
} catch (IOException ioe) {
if (VERBOSE) {
ioe.printStackTrace(System.out);
}
failure.clearDoFail();
- break;
+ assertTrue(writer.isClosed());
+ // Abort should have closed the deleter:
+ assertTrue(writer.deleter.isClosed());
+ break outer;
}
}
- assertEquals(20*(i+1)+extraCount, writer.numDocs());
}
- writer.close();
- IndexReader reader = DirectoryReader.open(directory);
- assertEquals(200+extraCount, reader.numDocs());
- reader.close();
+ assertFalse(DirectoryReader.indexExists(directory));
directory.close();
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Fri Dec 5 23:06:19 2014
@@ -905,6 +905,15 @@ public class TestIndexWriter extends Luc
// LUCENE-2239: won't work with NIOFS/MMAP
adder = new MockDirectoryWrapper(random, new RAMDirectory());
IndexWriterConfig conf = newIndexWriterConfig(random, new MockAnalyzer(random));
+ if (conf.getMergeScheduler() instanceof ConcurrentMergeScheduler) {
+ conf.setMergeScheduler(new SuppressingConcurrentMergeScheduler() {
+ @Override
+ protected boolean isOK(Throwable th) {
+ return th instanceof AlreadyClosedException ||
+ (th instanceof IllegalStateException && th.getMessage().contains("this writer hit an unrecoverable error"));
+ }
+ });
+ }
IndexWriter w = new IndexWriter(adder, conf);
Document doc = new Document();
doc.add(newStringField(random, "id", "500", Field.Store.NO));
@@ -968,6 +977,15 @@ public class TestIndexWriter extends Luc
}
IndexWriterConfig conf = newIndexWriterConfig(random,
new MockAnalyzer(random)).setMaxBufferedDocs(2);
+ if (conf.getMergeScheduler() instanceof ConcurrentMergeScheduler) {
+ conf.setMergeScheduler(new SuppressingConcurrentMergeScheduler() {
+ @Override
+ protected boolean isOK(Throwable th) {
+ return th instanceof AlreadyClosedException ||
+ (th instanceof IllegalStateException && th.getMessage().contains("this writer hit an unrecoverable error"));
+ }
+ });
+ }
//conf.setInfoStream(log);
w = new IndexWriter(dir, conf);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Fri Dec 5 23:06:19 2014
@@ -147,7 +147,7 @@ public class TestIndexWriterConfig exten
try {
assertNotNull(new RandomIndexWriter(random(), dir, conf));
fail("should have hit AlreadySetException");
- } catch (AlreadySetException e) {
+ } catch (IllegalStateException ise) {
// expected
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Fri Dec 5 23:06:19 2014
@@ -632,11 +632,9 @@ public class TestIndexWriterDelete exten
modifier.rollback();
}
- // If the close() succeeded, make sure there are
- // no unreferenced files.
+ // If the close() succeeded, make sure index is OK:
if (success) {
TestUtil.checkIndex(dir);
- TestIndexWriter.assertNoUnreferencedFiles(dir, "after writer.close");
}
dir.setRandomIOExceptionRate(randomIOExceptionRate);
dir.setMaxSizeInBytes(maxSizeInBytes);
@@ -922,8 +920,8 @@ public class TestIndexWriterDelete exten
break;
}
}
+ assertTrue(modifier.deleter.isClosed());
- modifier.close();
TestIndexWriter.assertNoUnreferencedFiles(dir, "docsWriter.abort() failed to delete unreferenced files");
dir.close();
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Fri Dec 5 23:06:19 2014
@@ -576,8 +576,7 @@ public class TestIndexWriterExceptions e
}
}
- // LUCENE-1072: make sure an errant exception on flushing
- // one segment only takes out those docs in that one flush
+ // make sure an aborting exception closes the writer:
public void testDocumentsWriterAbort() throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
FailOnlyOnFlush failure = new FailOnlyOnFlush();
@@ -590,20 +589,18 @@ public class TestIndexWriterExceptions e
String contents = "aa bb cc dd ee ff gg hh ii jj kk";
doc.add(newTextField("content", contents, Field.Store.NO));
boolean hitError = false;
- for(int i=0;i<200;i++) {
- try {
- writer.addDocument(doc);
- } catch (IOException ioe) {
- // only one flush should fail:
- assertFalse(hitError);
- hitError = true;
- }
+ writer.addDocument(doc);
+ try {
+ writer.addDocument(doc);
+ fail("did not hit exception");
+ } catch (IOException ioe) {
+ // only one flush should fail:
+ assertFalse(hitError);
+ hitError = true;
+ assertTrue(writer.deleter.isClosed());
+ assertTrue(writer.isClosed());
}
- assertTrue(hitError);
- writer.close();
- IndexReader reader = DirectoryReader.open(dir);
- assertEquals(198, reader.docFreq(new Term("content", "aa")));
- reader.close();
+ assertFalse(DirectoryReader.indexExists(dir));
dir.close();
}
@@ -1266,6 +1263,7 @@ public class TestIndexWriterExceptions e
new FailOnTermVectors(FailOnTermVectors.AFTER_INIT_STAGE),
new FailOnTermVectors(FailOnTermVectors.INIT_STAGE), };
int num = atLeast(1);
+ iters:
for (int j = 0; j < num; j++) {
for (FailOnTermVectors failure : failures) {
MockDirectoryWrapper dir = newMockDirectory();
@@ -1274,14 +1272,19 @@ public class TestIndexWriterExceptions e
int numDocs = 10 + random().nextInt(30);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
+ // random TV
Field field = newTextField(random(), "field", "a field", Field.Store.YES);
doc.add(field);
- // random TV
try {
w.addDocument(doc);
assertFalse(field.fieldType().storeTermVectors());
} catch (RuntimeException e) {
assertTrue(e.getMessage().startsWith(FailOnTermVectors.EXC_MSG));
+ // This is an aborting exception, so writer is closed:
+ assertTrue(w.deleter.isClosed());
+ assertTrue(w.isClosed());
+ dir.close();
+ continue iters;
}
if (random().nextInt(20) == 0) {
w.commit();
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java Fri Dec 5 23:06:19 2014
@@ -40,14 +40,15 @@ import org.apache.lucene.document.Sorted
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.TextField;
+import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.Rethrow;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.TestUtil;
/**
* Causes a bunch of non-aborting and aborting exceptions and checks that
@@ -63,6 +64,7 @@ public class TestIndexWriterExceptions2
if (dir instanceof MockDirectoryWrapper) {
((MockDirectoryWrapper)dir).setThrottling(MockDirectoryWrapper.Throttling.NEVER);
((MockDirectoryWrapper)dir).setUseSlowOpenClosers(false);
+ ((MockDirectoryWrapper)dir).setPreventDoubleWrite(false);
}
// log all exceptions we hit, in case we fail (for debugging)
@@ -96,11 +98,12 @@ public class TestIndexWriterExceptions2
// just for now, try to keep this test reproducible
conf.setMergeScheduler(new SerialMergeScheduler());
conf.setCodec(codec);
-
+
int numDocs = atLeast(500);
IndexWriter iw = new IndexWriter(dir, conf);
try {
+ boolean allowAlreadyClosed = false;
for (int i = 0; i < numDocs; i++) {
// TODO: add crankyDocValuesFields, etc
Document doc = new Document();
@@ -136,10 +139,21 @@ public class TestIndexWriterExceptions2
} else if (thingToDo == 2) {
iw.updateBinaryDocValue(new Term("id", Integer.toString(i)), "dv2", new BytesRef(Integer.toString(i+1)));
}
+ } catch (AlreadyClosedException ace) {
+ // OK: writer was closed by abort; we just reopen now:
+ assertTrue(iw.deleter.isClosed());
+ assertTrue(allowAlreadyClosed);
+ allowAlreadyClosed = false;
+ conf = newIndexWriterConfig(analyzer);
+ // just for now, try to keep this test reproducible
+ conf.setMergeScheduler(new SerialMergeScheduler());
+ conf.setCodec(codec);
+ iw = new IndexWriter(dir, conf);
} catch (Exception e) {
if (e.getMessage() != null && e.getMessage().startsWith("Fake IOException")) {
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
e.printStackTrace(exceptionStream);
+ allowAlreadyClosed = true;
} else {
Rethrow.rethrow(e);
}
@@ -159,10 +173,21 @@ public class TestIndexWriterExceptions2
if (random().nextBoolean()) {
iw.deleteDocuments(new Term("id", Integer.toString(i)), new Term("id", Integer.toString(-i)));
}
+ } catch (AlreadyClosedException ace) {
+ // OK: writer was closed by abort; we just reopen now:
+ assertTrue(iw.deleter.isClosed());
+ assertTrue(allowAlreadyClosed);
+ allowAlreadyClosed = false;
+ conf = newIndexWriterConfig(analyzer);
+ // just for now, try to keep this test reproducible
+ conf.setMergeScheduler(new SerialMergeScheduler());
+ conf.setCodec(codec);
+ iw = new IndexWriter(dir, conf);
} catch (Exception e) {
if (e.getMessage() != null && e.getMessage().startsWith("Fake IOException")) {
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
e.printStackTrace(exceptionStream);
+ allowAlreadyClosed = true;
} else {
Rethrow.rethrow(e);
}
@@ -186,10 +211,21 @@ public class TestIndexWriterExceptions2
if (DirectoryReader.indexExists(dir)) {
TestUtil.checkIndex(dir);
}
+ } catch (AlreadyClosedException ace) {
+ // OK: writer was closed by abort; we just reopen now:
+ assertTrue(iw.deleter.isClosed());
+ assertTrue(allowAlreadyClosed);
+ allowAlreadyClosed = false;
+ conf = newIndexWriterConfig(analyzer);
+ // just for now, try to keep this test reproducible
+ conf.setMergeScheduler(new SerialMergeScheduler());
+ conf.setCodec(codec);
+ iw = new IndexWriter(dir, conf);
} catch (Exception e) {
if (e.getMessage() != null && e.getMessage().startsWith("Fake IOException")) {
exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
e.printStackTrace(exceptionStream);
+ allowAlreadyClosed = true;
} else {
Rethrow.rethrow(e);
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Fri Dec 5 23:06:19 2014
@@ -57,6 +57,7 @@ public class TestIndexWriterOnDiskFull e
}
boolean doAbort = pass == 1;
long diskFree = TestUtil.nextInt(random(), 100, 300);
+ boolean indexExists = false;
while(true) {
if (VERBOSE) {
System.out.println("TEST: cycle: diskFree=" + diskFree);
@@ -82,6 +83,7 @@ public class TestIndexWriterOnDiskFull e
System.out.println("TEST: done adding docs; now commit");
}
writer.commit();
+ indexExists = true;
} catch (IOException e) {
if (VERBOSE) {
System.out.println("TEST: exception on addDoc");
@@ -118,9 +120,7 @@ public class TestIndexWriterOnDiskFull e
//_TestUtil.syncConcurrentMerges(ms);
- if (TestUtil.anyFilesExceptWriteLock(dir)) {
- assertNoUnreferencedFiles(dir, "after disk full during addDocument");
-
+ if (indexExists) {
// Make sure reader can open the index:
DirectoryReader.open(dir).close();
}
@@ -548,25 +548,9 @@ public class TestIndexWriterOnDiskFull e
writer.addDocument(doc);
fail("did not hit disk full");
} catch (IOException ioe) {
+ assertTrue(writer.deleter.isClosed());
+ assertTrue(writer.isClosed());
}
- // Without fix for LUCENE-1130: this call will hang:
- try {
- writer.addDocument(doc);
- fail("did not hit disk full");
- } catch (IOException ioe) {
- }
- try {
- writer.commit();
- fail("did not hit disk full");
- } catch (IOException ioe) {
- } finally {
- writer.close();
- }
-
- // Make sure once disk space is avail again, we can
- // cleanly close:
- dir.setMaxSizeInBytes(0);
- writer.close();
dir.close();
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java Fri Dec 5 23:06:19 2014
@@ -110,6 +110,9 @@ public class TestIndexWriterWithThreads
}
break;
}
+ } catch (AlreadyClosedException ace) {
+ // OK: abort closes the writer
+ break;
} catch (Throwable t) {
//t.printStackTrace(System.out);
if (noErrors) {
@@ -166,6 +169,9 @@ public class TestIndexWriterWithThreads
dir.setMaxSizeInBytes(0);
try {
writer.commit();
+ } catch (AlreadyClosedException ace) {
+ // OK: abort closes the writer
+ assertTrue(writer.deleter.isClosed());
} finally {
writer.close();
}
@@ -300,6 +306,9 @@ public class TestIndexWriterWithThreads
writer.commit();
writer.close();
success = true;
+ } catch (AlreadyClosedException ace) {
+ // OK: abort closes the writer
+ assertTrue(writer.deleter.isClosed());
} catch (IOException ioe) {
writer.rollback();
failure.clearDoFail();
@@ -329,10 +338,22 @@ public class TestIndexWriterWithThreads
public void _testSingleThreadFailure(MockDirectoryWrapper.Failure failure) throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
- .setMaxBufferedDocs(2)
- .setMergeScheduler(new ConcurrentMergeScheduler())
- .setCommitOnClose(false));
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
+ .setMaxBufferedDocs(2)
+ .setMergeScheduler(new ConcurrentMergeScheduler())
+ .setCommitOnClose(false);
+
+ if (iwc.getMergeScheduler() instanceof ConcurrentMergeScheduler) {
+ iwc.setMergeScheduler(new SuppressingConcurrentMergeScheduler() {
+ @Override
+ protected boolean isOK(Throwable th) {
+ return th instanceof AlreadyClosedException ||
+ (th instanceof IllegalStateException && th.getMessage().contains("this writer hit an unrecoverable error"));
+ }
+ });
+ }
+
+ IndexWriter writer = new IndexWriter(dir, iwc);
final Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_STORED);
customType.setStoreTermVectors(true);
@@ -353,11 +374,13 @@ public class TestIndexWriterWithThreads
} catch (IOException ioe) {
}
failure.clearDoFail();
- writer.addDocument(doc);
try {
+ writer.addDocument(doc);
writer.commit();
- } finally {
writer.close();
+ } catch (AlreadyClosedException ace) {
+ // OK: abort closes the writer
+ assertTrue(writer.deleter.isClosed());
}
dir.close();
}
Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java?rev=1643466&r1=1643465&r2=1643466&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java Fri Dec 5 23:06:19 2014
@@ -52,6 +52,7 @@ import org.apache.lucene.search.IndexSea
import org.apache.lucene.search.LiveFieldValues;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@@ -597,8 +598,21 @@ public class TestIDVersionPostingsFormat
} catch (IllegalArgumentException iae) {
// expected
}
+ try {
+ w.addDocument(doc);
+ fail("should have hit exc");
+ } catch (AlreadyClosedException ace) {
+ // expected
+ }
+ dir.close();
+ }
- doc = new Document();
+ public void testInvalidVersions2() throws IOException {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+ iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+ Document doc = new Document();
// Long.MAX_VALUE:
doc.add(new StringAndPayloadField("id", "id", new BytesRef(new byte[] {(byte)0x7f, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff})));
try {
@@ -608,7 +622,12 @@ public class TestIDVersionPostingsFormat
} catch (IllegalArgumentException iae) {
// expected
}
- w.close();
+ try {
+ w.addDocument(doc);
+ fail("should have hit exc");
+ } catch (AlreadyClosedException ace) {
+ // expected
+ }
dir.close();
}