LUCENE-5255: Make DocumentsWriter reference final in IW

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1530679 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2013-10-09 15:31:21 +00:00
parent f283429ee1
commit 3904328399
1 changed files with 4 additions and 23 deletions

View File

@ -228,7 +228,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
final SegmentInfos segmentInfos; // the segments
final FieldNumbers globalFieldNumberMap;
private DocumentsWriter docWriter;
private final DocumentsWriter docWriter;
private final Queue<Event> eventQueue;
final IndexFileDeleter deleter;
@ -1043,13 +1043,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
deleter.close();
}
// used by assert below
final DocumentsWriter oldWriter = docWriter;
synchronized (this) {
docWriter = null;
}
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "at close: " + segString());
}
@ -1061,7 +1054,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
synchronized(this) {
closed = true;
}
assert oldWriter.perThreadPool.numDeactivatedThreadStates() == oldWriter.perThreadPool.getMaxThreadStates() : "" + oldWriter.perThreadPool.numDeactivatedThreadStates() + " " + oldWriter.perThreadPool.getMaxThreadStates();
assert docWriter.perThreadPool.numDeactivatedThreadStates() == docWriter.perThreadPool.getMaxThreadStates() : "" + docWriter.perThreadPool.numDeactivatedThreadStates() + " " + docWriter.perThreadPool.getMaxThreadStates();
} catch (OutOfMemoryError oom) {
handleOOM(oom, "closeInternal");
} finally {
@ -1096,14 +1089,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
* @see #numDocs */
public synchronized int maxDoc() {
ensureOpen();
int count;
if (docWriter != null)
count = docWriter.getNumDocs();
else
count = 0;
count += segmentInfos.totalDocCount();
return count;
return docWriter.getNumDocs() + segmentInfos.totalDocCount();
}
/** Returns total number of docs in this index, including
@ -1114,12 +1100,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
* @see #numDocs */
public synchronized int numDocs() {
ensureOpen();
int count;
if (docWriter != null)
count = docWriter.getNumDocs();
else
count = 0;
int count = docWriter.getNumDocs();
for (final SegmentInfoPerCommit info : segmentInfos) {
count += info.info.getDocCount() - numDeletedDocs(info);
}