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/07 12:37:35 UTC
svn commit: r1643662 [2/6] - in /lucene/dev/branches/lucene6005: ./ lucene/
lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/
lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/
lucene/core/src/java/org/apache/lucene/co...
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Sun Dec 7 11:37:32 2014
@@ -37,6 +37,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;
/**
@@ -214,7 +215,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")) {
@@ -225,18 +225,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);
}
}
}
@@ -250,15 +249,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 {
@@ -272,15 +269,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 {
@@ -351,7 +347,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) {
@@ -383,7 +379,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);
@@ -408,7 +404,7 @@ final class DocumentsWriter implements C
}
boolean updateDocuments(final Iterable<? extends Iterable<? extends IndexableField>> docs,
- final Term delTerm) throws IOException {
+ final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
final ThreadState perThread = flushControl.obtainAndLock();
@@ -425,18 +421,15 @@ final class DocumentsWriter implements C
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocuments(docs, 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);
@@ -447,7 +440,7 @@ final class DocumentsWriter implements C
return postUpdate(flushingDWPT, hasEvents);
}
- boolean updateDocument(final Iterable<? extends IndexableField> doc, final Term delTerm) throws IOException {
+ boolean updateDocument(final Iterable<? extends IndexableField> doc, final Term delTerm) throws IOException, AbortingException {
boolean hasEvents = preUpdate();
@@ -465,18 +458,15 @@ final class DocumentsWriter implements C
final int dwptNumDocs = dwpt.getNumDocsInRAM();
try {
dwpt.updateDocument(doc, 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);
@@ -487,12 +477,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: "
@@ -558,7 +549,6 @@ final class DocumentsWriter implements C
}
} finally {
flushControl.doAfterFlush(flushingDWPT);
- flushingDWPT.checkAndResetHasAborted();
}
flushingDWPT = flushControl.nextPendingFlush();
@@ -607,8 +597,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");
@@ -641,7 +631,7 @@ final class DocumentsWriter implements C
}
ticketQueue.addDeletes(flushingDeleteQueue);
}
- ticketQueue.forcePurge(indexWriter);
+ ticketQueue.forcePurge(writer);
assert !flushingDeleteQueue.anyChanges() && !ticketQueue.hasTickets();
} finally {
assert flushingDeleteQueue == currentFullFlushDelQueue;
@@ -659,9 +649,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/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Sun Dec 7 11:37:32 2014
@@ -310,7 +310,7 @@ final class DocumentsWriterFlushControl
dwpt = perThreadPool.reset(perThread, closed);
numPending--;
blockedFlushes.add(new BlockedFlush(dwpt, bytes));
- }finally {
+ } finally {
perThread.unlock();
}
}
@@ -611,20 +611,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 {
@@ -636,7 +636,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/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Dec 7 11:37:32 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.text.NumberFormat;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Locale;
import java.util.Set;
@@ -108,9 +109,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");
@@ -121,9 +122,7 @@ class DocumentsWriterPerThread {
}
pendingUpdates.clear();
- createdFiles.addAll(directory.getCreatedFiles());
} finally {
- aborting = false;
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "done abort");
}
@@ -141,8 +140,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;
@@ -180,7 +178,7 @@ class DocumentsWriterPerThread {
pendingUpdates.clear();
deleteSlice = deleteQueue.newSlice();
- segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, null, StringHelper.randomId());
+ segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);
@@ -190,20 +188,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);
@@ -220,7 +208,7 @@ class DocumentsWriterPerThread {
}
}
- public void updateDocument(Iterable<? extends IndexableField> doc, Term delTerm) throws IOException {
+ public void updateDocument(Iterable<? extends IndexableField> doc, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocument start");
if (doc instanceof Document) {
Document doc2 = (Document) doc;
@@ -251,19 +239,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, Term delTerm) throws IOException {
+ public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Term delTerm) throws IOException, AbortingException {
testPoint("DocumentsWriterPerThread addDocuments start");
assert deleteQueue != null;
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
@@ -297,14 +281,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);
@@ -321,7 +300,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;
@@ -414,7 +393,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);
@@ -435,7 +414,7 @@ class DocumentsWriterPerThread {
pendingUpdates.docIDs.clear();
}
- if (aborting) {
+ if (aborted) {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "flush: skip because aborting is set");
}
@@ -446,8 +425,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();
@@ -487,13 +464,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);
}
}
@@ -614,7 +589,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/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sun Dec 7 11:37:32 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/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sun Dec 7 11:37:32 2014
@@ -416,7 +416,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.
@@ -434,8 +434,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 {
@@ -1208,8 +1208,8 @@ public class IndexWriter implements Clos
}
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "updateDocuments");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "updateDocuments");
}
}
@@ -1358,8 +1358,8 @@ public class IndexWriter implements Clos
}
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "updateDocument");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "updateDocument");
}
}
@@ -2503,7 +2503,7 @@ public class IndexWriter implements Clos
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
SegmentInfo info = new SegmentInfo(directory, Version.LATEST, mergedName, -1,
- false, codec, null, StringHelper.randomId());
+ false, codec, null, StringHelper.randomId(), new HashMap<>());
SegmentMerger merger = new SegmentMerger(fieldTypes, mergeReaders, info, infoStream, trackingDir,
MergeState.CheckAbort.NONE, globalFieldNumberMap,
@@ -2599,7 +2599,7 @@ public class IndexWriter implements Clos
// Same SI as before but we change directory and name
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
info.info.getUseCompoundFile(), info.info.getCodec(),
- info.info.getDiagnostics(), info.info.getId());
+ info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
info.getFieldInfosGen(), info.getDocValuesGen());
@@ -2691,7 +2691,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.
@@ -2737,8 +2737,8 @@ public class IndexWriter implements Clos
doAfterFlush();
}
}
- } catch (OutOfMemoryError oom) {
- tragicEvent(oom, "prepareCommit");
+ } catch (AbortingException | OutOfMemoryError tragedy) {
+ tragicEvent(tragedy, "prepareCommit");
}
boolean success = false;
@@ -2983,7 +2983,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);
@@ -3000,8 +3000,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 {
@@ -3719,7 +3719,7 @@ public class IndexWriter implements Clos
// ConcurrentMergePolicy we keep deterministic segment
// names.
final String mergeSegmentName = newSegmentName();
- SegmentInfo si = new SegmentInfo(directory, Version.LATEST, mergeSegmentName, -1, false, codec, null, StringHelper.randomId());
+ SegmentInfo si = new SegmentInfo(directory, Version.LATEST, mergeSegmentName, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
Map<String,String> details = new HashMap<>();
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
@@ -4232,7 +4232,9 @@ public class IndexWriter implements Clos
synchronized(this) {
long fieldTypesChangeCount = fieldTypes.getChangeCount();
- assert lastCommitChangeCount <= changeCount: "lastCommitChangeCount=" + lastCommitChangeCount + " changeCount=" + changeCount;
+ if (lastCommitChangeCount > changeCount) {
+ throw new IllegalStateException("lastCommitChangeCount=" + lastCommitChangeCount + ",changeCount=" + changeCount);
+ }
assert lastCommitFieldTypesChangeCount <= fieldTypesChangeCount: "lastCommitFieldTypesChangeCount=" + lastCommitFieldTypesChangeCount + " fieldTypesChangeCount=" + fieldTypesChangeCount;
if (pendingCommitChangeCount == lastCommitChangeCount && pendingCommitFieldTypesChangeCount == lastCommitFieldTypesChangeCount) {
@@ -4361,19 +4363,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) {
@@ -4385,7 +4396,8 @@ public class IndexWriter implements Clos
}
}
}
- IOUtils.reThrowUnchecked(tragedy);
+
+ IOUtils.reThrow(tragedy);
}
// Used for testing. Current points:
@@ -4550,11 +4562,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/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Sun Dec 7 11:37:32 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/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Sun Dec 7 11:37:32 2014
@@ -209,8 +209,8 @@ public abstract class MergePolicy {
while (paused) {
try {
// In theory we could wait() indefinitely, but we
- // do 1000 msec, defensively
- wait(1000);
+ // do 250 msec, defensively
+ wait(250);
} catch (InterruptedException ie) {
throw new RuntimeException(ie);
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Sun Dec 7 11:37:32 2014
@@ -21,8 +21,10 @@ package org.apache.lucene.index;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import java.util.regex.Matcher;
@@ -65,6 +67,8 @@ public final class SegmentInfo {
private Codec codec;
private Map<String,String> diagnostics;
+
+ private Map<String,String> attributes;
// Tracks the Lucene version this segment was created with, since 3.1. Null
// indicates an older than 3.0 index, and it's used to detect a too old index.
@@ -90,7 +94,7 @@ public final class SegmentInfo {
*/
public SegmentInfo(Directory dir, Version version, String name, int docCount,
boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
- byte[] id) {
+ byte[] id, Map<String,String> attributes) {
assert !(dir instanceof TrackingDirectoryWrapper);
this.dir = dir;
this.version = version;
@@ -103,6 +107,7 @@ public final class SegmentInfo {
if (id.length != StringHelper.ID_LENGTH) {
throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));
}
+ this.attributes = Objects.requireNonNull(attributes);
}
/**
@@ -267,4 +272,34 @@ public final class SegmentInfo {
String namedForThisSegment(String file) {
return name + IndexFileNames.stripSegmentName(file);
}
+
+ /**
+ * Get a codec attribute value, or null if it does not exist
+ */
+ public String getAttribute(String key) {
+ return attributes.get(key);
+ }
+
+ /**
+ * Puts a codec attribute value.
+ * <p>
+ * This is a key-value mapping for the field that the codec can use to store
+ * additional metadata, and will be available to the codec when reading the
+ * segment via {@link #getAttribute(String)}
+ * <p>
+ * If a value already exists for the field, it will be replaced with the new
+ * value.
+ */
+ public String putAttribute(String key, String value) {
+ return attributes.put(key, value);
+ }
+
+ /**
+ * Returns the internal codec attributes map.
+ * @return internal codec attributes map.
+ */
+ public Map<String,String> getAttributes() {
+ return attributes;
+ }
}
+
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java Sun Dec 7 11:37:32 2014
@@ -126,7 +126,7 @@ final class TermVectorsConsumer extends
super.abort();
} finally {
if (writer != null) {
- writer.abort();
+ IOUtils.closeWhileHandlingException(writer);
writer = null;
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java Sun Dec 7 11:37:32 2014
@@ -106,6 +106,25 @@ public final class SearcherManager exten
current = getSearcher(searcherFactory, DirectoryReader.open(dir));
}
+ /**
+ * Creates and returns a new SearcherManager from an existing {@link DirectoryReader}. Note that
+ * this steals the incoming reference.
+ *
+ * @param reader the DirectoryReader.
+ * @param searcherFactory An optional {@link SearcherFactory}. Pass
+ * <code>null</code> if you don't require the searcher to be warmed
+ * before going live or other custom behavior.
+ *
+ * @throws IOException if there is a low-level I/O error
+ */
+ public SearcherManager(DirectoryReader reader, SearcherFactory searcherFactory) throws IOException {
+ if (searcherFactory == null) {
+ searcherFactory = new SearcherFactory();
+ }
+ this.searcherFactory = searcherFactory;
+ this.current = getSearcher(searcherFactory, reader);
+ }
+
@Override
protected void decRef(IndexSearcher reference) throws IOException {
reference.getIndexReader().decRef();
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/UsageTrackingFilterCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/UsageTrackingFilterCachingPolicy.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/UsageTrackingFilterCachingPolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/UsageTrackingFilterCachingPolicy.java Sun Dec 7 11:37:32 2014
@@ -36,18 +36,18 @@ import org.apache.lucene.util.FrequencyT
*/
public final class UsageTrackingFilterCachingPolicy implements FilterCachingPolicy {
- private static boolean isCostly(Filter filter) {
+ static boolean isCostly(Filter filter) {
// This does not measure the cost of iterating over the filter (for this we
// already have the DocIdSetIterator#cost API) but the cost to build the
// DocIdSet in the first place
return filter instanceof MultiTermQueryWrapperFilter;
}
- private static boolean isCheapToCache(DocIdSet set) {
+ static boolean isCheapToCache(DocIdSet set) {
// the produced doc set is already cacheable, so caching has no
// overhead at all. TODO: extend this to sets whose iterators have a low
// cost?
- return set.isCacheable();
+ return set == null || set.isCacheable();
}
private final FilterCachingPolicy.CacheOnLargeSegments segmentPolicy;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Sun Dec 7 11:37:32 2014
@@ -282,7 +282,7 @@ public abstract class FSDirectory extend
private final String name;
public FSIndexOutput(String name) throws IOException {
- super(new FilterOutputStream(Files.newOutputStream(directory.resolve(name))) {
+ super("FSIndexOutput(path=\"" + directory.resolve(name) + "\")", new FilterOutputStream(Files.newOutputStream(directory.resolve(name))) {
// This implementation ensures, that we never write more than CHUNK_SIZE bytes:
@Override
public void write(byte[] b, int offset, int length) throws IOException {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Sun Dec 7 11:37:32 2014
@@ -31,6 +31,17 @@ import java.io.IOException;
*/
public abstract class IndexOutput extends DataOutput implements Closeable {
+ private final String resourceDescription;
+
+ /** Sole constructor. resourceDescription should be non-null, opaque string
+ * describing this resource; it's returned from {@link #toString}. */
+ protected IndexOutput(String resourceDescription) {
+ if (resourceDescription == null) {
+ throw new IllegalArgumentException("resourceDescription must not be null");
+ }
+ this.resourceDescription = resourceDescription;
+ }
+
/** Closes this stream to further operations. */
@Override
public abstract void close() throws IOException;
@@ -42,4 +53,9 @@ public abstract class IndexOutput extend
/** Returns the current checksum of bytes written so far */
public abstract long getChecksum() throws IOException;
+
+ @Override
+ public String toString() {
+ return resourceDescription;
+ }
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java Sun Dec 7 11:37:32 2014
@@ -36,7 +36,8 @@ public class OutputStreamIndexOutput ext
* @param bufferSize the buffer size in bytes used to buffer writes internally.
* @throws IllegalArgumentException if the given buffer size is less or equal to <tt>0</tt>
*/
- public OutputStreamIndexOutput(OutputStream out, int bufferSize) {
+ public OutputStreamIndexOutput(String resourceDescription, OutputStream out, int bufferSize) {
+ super(resourceDescription);
this.os = new BufferedOutputStream(new CheckedOutputStream(out, crc), bufferSize);
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Sun Dec 7 11:37:32 2014
@@ -173,7 +173,7 @@ public class RAMDirectory extends BaseDi
existing.directory = null;
}
fileMap.put(name, file);
- return new RAMOutputStream(file, true);
+ return new RAMOutputStream(name, file, true);
}
/**
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Sun Dec 7 11:37:32 2014
@@ -46,10 +46,17 @@ public class RAMOutputStream extends Ind
/** Construct an empty output buffer. */
public RAMOutputStream() {
- this(new RAMFile(), false);
+ this("noname", new RAMFile(), false);
}
+ /** Creates this, with no name. */
public RAMOutputStream(RAMFile f, boolean checksum) {
+ this("noname", f, checksum);
+ }
+
+ /** Creates this, with specified name. */
+ public RAMOutputStream(String name, RAMFile f, boolean checksum) {
+ super("RAMOutputStream(name=\"" + name + "\")");
file = f;
// make sure that we switch to the
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java Sun Dec 7 11:37:32 2014
@@ -37,6 +37,7 @@ final class RateLimitedIndexOutput exten
private long currentMinPauseCheckBytes;
RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
+ super("RateLimitedIndexOutput(" + delegate + ")");
this.delegate = delegate;
this.rateLimiter = rateLimiter;
this.currentMinPauseCheckBytes = rateLimiter.getMinPauseCheckBytes();
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Sun Dec 7 11:37:32 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.document.Docume
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;
@@ -61,18 +62,11 @@ public class TestCompressingStoredFields
// make sure that #writeField will fail to trigger an abort
Document invalidDoc = iw.newDocument();
-
+
+ // nocommit this no longer aborts....
try {
invalidDoc.addStored("invalid", (String) null);
} finally {
- 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.close();
dir.close();
}
Copied: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (from r1643659, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java&r1=1643659&r2=1643662&rev=1643662&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java Sun Dec 7 11:37:32 2014
@@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene5
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.document.Document;
-import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
@@ -45,9 +44,9 @@ public class TestLucene50StoredFieldsFor
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene50Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
- Document doc = new Document();
- doc.add(new StoredField("field1", "value1"));
- doc.add(new StoredField("field2", "value2"));
+ Document doc = iw.newDocument();
+ doc.addStored("field1", "value1");
+ doc.addStored("field2", "value2");
iw.addDocument(doc);
if (random().nextInt(4) == 0) {
iw.forceMerge(1);
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Sun Dec 7 11:37:32 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Random;
@@ -218,7 +219,7 @@ public class TestCodecs extends LuceneTe
final FieldInfos fieldInfos = builder.finish();
final Directory dir = newDirectory();
Codec codec = Codec.getDefault();
- final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
+ final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
this.write(si, fieldInfos, dir, fields);
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@@ -275,7 +276,7 @@ public class TestCodecs extends LuceneTe
}
Codec codec = Codec.getDefault();
- final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
+ final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
this.write(si, fieldInfos, dir, fields);
if (VERBOSE) {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Sun Dec 7 11:37:32 2014
@@ -27,6 +27,7 @@ import org.apache.lucene.analysis.MockAn
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldTypes;
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;
@@ -79,11 +80,20 @@ public class TestConcurrentMergeSchedule
MockDirectoryWrapper directory = newMockDirectory();
FailOnlyOnFlush failure = new FailOnlyOnFlush();
directory.failOn(failure);
+ 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);
- IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random()))
- .setMaxBufferedDocs(2));
- int extraCount = 0;
-
+ outer:
for(int i=0;i<10;i++) {
if (VERBOSE) {
System.out.println("TEST: iter=" + i);
@@ -108,22 +118,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();
}
@@ -475,4 +483,72 @@ public class TestConcurrentMergeSchedule
w.close();
dir.close();
}
+
+ // LUCENE-6094
+ public void testHangDuringRollback() throws Throwable {
+ Directory dir = newMockDirectory();
+ IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+ iwc.setMaxBufferedDocs(2);
+ LogDocMergePolicy mp = new LogDocMergePolicy();
+ iwc.setMergePolicy(mp);
+ mp.setMergeFactor(2);
+ final CountDownLatch mergeStart = new CountDownLatch(1);
+ final CountDownLatch mergeFinish = new CountDownLatch(1);
+ ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler() {
+ @Override
+ protected void doMerge(MergePolicy.OneMerge merge) throws IOException {
+ mergeStart.countDown();
+ try {
+ mergeFinish.await();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ super.doMerge(merge);
+ }
+ };
+ cms.setMaxMergesAndThreads(1, 1);
+ iwc.setMergeScheduler(cms);
+
+ final IndexWriter w = new IndexWriter(dir, iwc);
+
+ w.addDocument(w.newDocument());
+ w.addDocument(w.newDocument());
+ // flush
+
+ w.addDocument(w.newDocument());
+ w.addDocument(w.newDocument());
+ // flush + merge
+
+ // Wait for merge to kick off
+ mergeStart.await();
+
+ new Thread() {
+ @Override
+ public void run() {
+ try {
+ w.addDocument(w.newDocument());
+ w.addDocument(w.newDocument());
+ // flush
+
+ w.addDocument(w.newDocument());
+ // W/o the fix for LUCENE-6094 we would hang forever here:
+ w.addDocument(w.newDocument());
+ // flush + merge
+
+ // Now allow first merge to finish:
+ mergeFinish.countDown();
+
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }.start();
+
+ while (w.numDocs() != 8) {
+ Thread.sleep(10);
+ }
+
+ w.rollback();
+ dir.close();
+ }
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java Sun Dec 7 11:37:32 2014
@@ -118,6 +118,9 @@ public class TestDemoParallelLeafReader
IndexWriterConfig iwc = getIndexWriterConfig();
iwc.setMergePolicy(new ReindexingMergePolicy(iwc.getMergePolicy()));
+ if (DEBUG) {
+ System.out.println("TEST: use IWC:\n" + iwc);
+ }
w = new IndexWriter(indexDir, iwc);
w.getConfig().setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
@@ -657,7 +660,12 @@ public class TestDemoParallelLeafReader
return new ReindexingReader(root) {
@Override
protected IndexWriterConfig getIndexWriterConfig() throws IOException {
- return newIndexWriterConfig();
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ TieredMergePolicy tmp = new TieredMergePolicy();
+ // We write tiny docs, so we need tiny floor to avoid O(N^2) merging:
+ tmp.setFloorSegmentMB(.01);
+ iwc.setMergePolicy(tmp);
+ return iwc;
}
@Override
@@ -706,7 +714,12 @@ public class TestDemoParallelLeafReader
return new ReindexingReader(root) {
@Override
protected IndexWriterConfig getIndexWriterConfig() throws IOException {
- return newIndexWriterConfig();
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ TieredMergePolicy tmp = new TieredMergePolicy();
+ // We write tiny docs, so we need tiny floor to avoid O(N^2) merging:
+ tmp.setFloorSegmentMB(.01);
+ iwc.setMergePolicy(tmp);
+ return iwc;
}
@Override
@@ -792,7 +805,12 @@ public class TestDemoParallelLeafReader
return new ReindexingReader(root) {
@Override
protected IndexWriterConfig getIndexWriterConfig() throws IOException {
- return newIndexWriterConfig();
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ TieredMergePolicy tmp = new TieredMergePolicy();
+ // We write tiny docs, so we need tiny floor to avoid O(N^2) merging:
+ tmp.setFloorSegmentMB(.01);
+ iwc.setMergePolicy(tmp);
+ return iwc;
}
@Override
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Sun Dec 7 11:37:32 2014
@@ -28,6 +28,7 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collection;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@@ -223,7 +224,7 @@ public class TestDoc extends LuceneTestC
final Codec codec = Codec.getDefault();
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
- final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId());
+ final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
SegmentMerger merger = new SegmentMerger(fieldTypes, Arrays.<LeafReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir,
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Sun Dec 7 11:37:32 2014
@@ -799,6 +799,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);
FieldTypes fieldTypes = w.getFieldTypes();
fieldTypes.setMultiValued("sortedsetdv");
@@ -869,6 +878,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);
FieldTypes fieldTypes = w.getFieldTypes();
@@ -2324,7 +2342,7 @@ public class TestIndexWriter extends Luc
MockDirectoryWrapper dir = newMockDirectory();
if (TestUtil.isWindowsFS(dir)) {
dir.close();
- assumeFalse("this test can't run on Windows", true);
+ assumeFalse("this test can't run on simulated windows (WindowsFS)", true);
}
// don't act like windows either, or the test won't simulate the condition
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Sun Dec 7 11:37:32 2014
@@ -146,7 +146,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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Sun Dec 7 11:37:32 2014
@@ -643,11 +643,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);
@@ -928,8 +926,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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Sun Dec 7 11:37:32 2014
@@ -544,8 +544,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();
@@ -558,20 +557,18 @@ public class TestIndexWriterExceptions e
String contents = "aa bb cc dd ee ff gg hh ii jj kk";
doc.addLargeText("content", contents);
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();
}
@@ -1250,6 +1247,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();
@@ -1260,12 +1258,16 @@ public class TestIndexWriterExceptions e
for (int i = 0; i < numDocs; i++) {
Document doc = w.newDocument();
doc.addLargeText("field", "a field");
- // random TV
try {
w.addDocument(doc);
assertFalse(fieldTypes.getTermVectors("field"));
} 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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java Sun Dec 7 11:37:32 2014
@@ -32,10 +32,12 @@ import org.apache.lucene.codecs.assertin
import org.apache.lucene.codecs.cranky.CrankyCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldTypes;
+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.Rethrow;
import org.apache.lucene.util.TestUtil;
@@ -55,6 +57,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)
@@ -88,7 +91,7 @@ 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);
@@ -100,9 +103,31 @@ public class TestIndexWriterExceptions2
fieldTypes.setMultiValued("stored1");
try {
+ boolean allowAlreadyClosed = false;
for (int i = 0; i < numDocs; i++) {
// TODO: add crankyDocValuesFields, etc
- Document doc = iw.newDocument();
+ Document doc;
+ try {
+ doc = iw.newDocument();
+ } 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);
+ fieldTypes = iw.getFieldTypes();
+ fieldTypes.enableTermVectors("text_vectors");
+ fieldTypes.disableSorting("dv2");
+ fieldTypes.setMultiValued("dv4");
+ fieldTypes.setMultiValued("dv5");
+ fieldTypes.setMultiValued("stored1");
+ continue;
+ }
+
doc.addAtom("id", Integer.toString(i));
doc.addInt("dv", i);
doc.addBinary("dv2", new BytesRef(Integer.toString(i)));
@@ -133,10 +158,28 @@ 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);
+ fieldTypes = iw.getFieldTypes();
+ fieldTypes.enableTermVectors("text_vectors");
+ fieldTypes.disableSorting("dv2");
+ fieldTypes.setMultiValued("dv4");
+ fieldTypes.setMultiValued("dv5");
+ fieldTypes.setMultiValued("stored1");
+
} 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);
}
@@ -156,10 +199,27 @@ 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);
+ fieldTypes = iw.getFieldTypes();
+ fieldTypes.enableTermVectors("text_vectors");
+ fieldTypes.disableSorting("dv2");
+ fieldTypes.setMultiValued("dv4");
+ fieldTypes.setMultiValued("dv5");
+ fieldTypes.setMultiValued("stored1");
} 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);
}
@@ -183,10 +243,27 @@ 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);
+ fieldTypes = iw.getFieldTypes();
+ fieldTypes.enableTermVectors("text_vectors");
+ fieldTypes.disableSorting("dv2");
+ fieldTypes.setMultiValued("dv4");
+ fieldTypes.setMultiValued("dv5");
+ fieldTypes.setMultiValued("stored1");
} 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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Sun Dec 7 11:37:32 2014
@@ -53,6 +53,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);
@@ -78,6 +79,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");
@@ -114,9 +116,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();
}
@@ -542,25 +542,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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java Sun Dec 7 11:37:32 2014
@@ -106,6 +106,9 @@ public class TestIndexWriterWithThreads
}
break;
}
+ } catch (AlreadyClosedException ace) {
+ // OK: abort closes the writer
+ break;
} catch (Throwable t) {
//t.printStackTrace(System.out);
if (noErrors) {
@@ -162,6 +165,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();
}
@@ -296,6 +302,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();
@@ -325,10 +334,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);
FieldTypes fieldTypes = writer.getFieldTypes();
fieldTypes.enableTermVectors("field");
fieldTypes.enableTermVectorOffsets("field");
@@ -350,11 +371,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/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Sun Dec 7 11:37:32 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
+import java.util.HashMap;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
@@ -79,7 +80,7 @@ public class TestSegmentMerger extends L
public void testMerge() throws IOException {
final Codec codec = Codec.getDefault();
- final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId());
+ final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
FieldTypes fieldTypes = FieldTypes.getFieldTypes(merge1Dir, new MockAnalyzer(random()));
SegmentMerger merger = new SegmentMerger(fieldTypes, Arrays.<LeafReader>asList(reader1, reader2),
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java Sun Dec 7 11:37:32 2014
@@ -47,6 +47,7 @@ public class TestTermVectorsWriter exten
fieldTypes.setAnalyzerOffsetGap("field", 0);
fieldTypes.setAnalyzerPositionGap("field", 0);
fieldTypes.setDocValuesType("field", DocValuesType.NONE);
+ fieldTypes.disableFastRanges("field");
fieldTypes.enableTermVectors("field");
fieldTypes.enableTermVectorOffsets("field");
fieldTypes.enableTermVectorPositions("field");
Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/mockfile/TestMockFilesystems.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/mockfile/TestMockFilesystems.java?rev=1643662&r1=1643661&r2=1643662&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/mockfile/TestMockFilesystems.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/mockfile/TestMockFilesystems.java Sun Dec 7 11:37:32 2014
@@ -27,6 +27,7 @@ import java.nio.channels.FileChannel;
import java.nio.channels.SeekableByteChannel;
import java.nio.file.FileSystem;
import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
@@ -211,6 +212,30 @@ public class TestMockFilesystems extends
file.close();
}
+ public void testVerboseFSNoSuchFileException() throws IOException {
+ Path dir = FilterPath.unwrap(createTempDir());
+ FileSystem fs = new VerboseFS(dir.getFileSystem(), InfoStream.NO_OUTPUT).getFileSystem(URI.create("file:///"));
+ Path wrapped = new FilterPath(dir, fs);
+ try {
+ AsynchronousFileChannel.open(wrapped.resolve("doesNotExist.rip"));
+ fail("did not hit exception");
+ } catch (NoSuchFileException nsfe) {
+ // expected
+ }
+ try {
+ FileChannel.open(wrapped.resolve("doesNotExist.rip"));
+ fail("did not hit exception");
+ } catch (NoSuchFileException nsfe) {
+ // expected
+ }
+ try {
+ Files.newByteChannel(wrapped.resolve("stillopen"));
+ fail("did not hit exception");
+ } catch (NoSuchFileException nsfe) {
+ // expected
+ }
+ }
+
public void testTooManyOpenFiles() throws IOException {
int n = 60;
Copied: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java (from r1643659, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java&r1=1643659&r2=1643662&rev=1643662&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java Sun Dec 7 11:37:32 2014
@@ -34,7 +34,7 @@ public class TestFilterCachingPolicy ext
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
final int numDocs = atLeast(100);
for (int i = 0; i < numDocs; ++i) {
- w.addDocument(new Document());
+ w.addDocument(w.newDocument());
}
final IndexReader reader = w.getReader();
for (float minSizeRatio : new float[] {Float.MIN_VALUE, 0.01f, 0.1f, 0.9f}) {