mirror of https://github.com/apache/lucene.git
Commit LUCENE-3598 to not corrumpt Robert's work on Default*: Improve InfoStream class in trunk to be more consistent with logging-frameworks like slf4j/log4j/commons-logging
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1209059 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
814d35e437
commit
21989c90c2
|
@ -88,7 +88,7 @@ class BufferedDeletesStream {
|
|||
deletes.add(packet);
|
||||
numTerms.addAndGet(packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(packet.bytesUsed);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get());
|
||||
}
|
||||
assert checkDeleteStats();
|
||||
|
@ -159,13 +159,13 @@ class BufferedDeletesStream {
|
|||
assert checkDeleteStats();
|
||||
|
||||
if (!any()) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: no deletes; skipping");
|
||||
}
|
||||
return new ApplyDeletesResult(false, nextGen++, null);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
|
||||
}
|
||||
|
||||
|
@ -237,7 +237,7 @@ class BufferedDeletesStream {
|
|||
allDeleted.add(info);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
|
||||
|
@ -279,7 +279,7 @@ class BufferedDeletesStream {
|
|||
allDeleted.add(info);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
}
|
||||
|
@ -290,7 +290,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes took " + (System.currentTimeMillis()-t0) + " msec");
|
||||
}
|
||||
// assert infos != segmentInfos || !any() : "infos=" + infos + " segmentInfos=" + segmentInfos + " any=" + any;
|
||||
|
@ -313,7 +313,7 @@ class BufferedDeletesStream {
|
|||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size());
|
||||
}
|
||||
final int limit = deletes.size();
|
||||
|
@ -333,7 +333,7 @@ class BufferedDeletesStream {
|
|||
|
||||
private synchronized void prune(int count) {
|
||||
if (count > 0) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain");
|
||||
}
|
||||
for(int delIDX=0;delIDX<count;delIDX++) {
|
||||
|
|
|
@ -218,7 +218,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
* </pre>
|
||||
*/
|
||||
protected boolean verbose() {
|
||||
return writer != null && writer.verbose();
|
||||
return writer != null && writer.infoStream.isEnabled("CMS");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -226,10 +226,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
* called and returned true.
|
||||
*/
|
||||
protected void message(String message) {
|
||||
final InfoStream infoStream = writer.infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("CMS", message);
|
||||
}
|
||||
writer.infoStream.message("CMS", message);
|
||||
}
|
||||
|
||||
private synchronized void initMergeThreadPriority() {
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.store.LockObtainFailedException;
|
|||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/**
|
||||
|
@ -685,9 +686,10 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
segmentInfos.setUserData(commitUserData);
|
||||
// Default deleter (for backwards compatibility) is
|
||||
// KeepOnlyLastCommitDeleter:
|
||||
// nocommit: Decide what to do with InfoStream here? Use default or keep NO_OUTPUT?
|
||||
IndexFileDeleter deleter = new IndexFileDeleter(directory,
|
||||
deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
|
||||
segmentInfos, null, null);
|
||||
segmentInfos, InfoStream.NO_OUTPUT, null);
|
||||
segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
|
||||
segmentInfos.changed();
|
||||
|
||||
|
|
|
@ -216,9 +216,7 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("DW", "abort");
|
||||
}
|
||||
infoStream.message("DW", "abort");
|
||||
|
||||
final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
|
||||
while (threadsIterator.hasNext()) {
|
||||
|
@ -243,14 +241,14 @@ final class DocumentsWriter {
|
|||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "done abort; abortedFiles=" + abortedFiles + " success=" + success);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
boolean anyChanges() {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get()
|
||||
+ " deletes=" + anyDeletions() + " hasTickets:"
|
||||
+ ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
|
||||
|
@ -289,7 +287,7 @@ final class DocumentsWriter {
|
|||
boolean maybeMerge = false;
|
||||
if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
|
||||
// Help out flushing any queued DWPTs so we can un-stall:
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
|
||||
}
|
||||
do {
|
||||
|
@ -300,14 +298,14 @@ final class DocumentsWriter {
|
|||
maybeMerge |= doFlush(flushingDWPT);
|
||||
}
|
||||
|
||||
if (infoStream != null && flushControl.anyStalledThreads()) {
|
||||
if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
|
||||
infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
|
||||
}
|
||||
|
||||
flushControl.waitIfStalled(); // block if stalled
|
||||
} while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "continue indexing after helping out flushing DocumentsWriter is healthy");
|
||||
}
|
||||
}
|
||||
|
@ -466,7 +464,7 @@ final class DocumentsWriter {
|
|||
final double ramBufferSizeMB = indexWriter.getConfig().getRAMBufferSizeMB();
|
||||
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
|
||||
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "force apply deletes bytesUsed=" + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*ramBufferSizeMB));
|
||||
}
|
||||
applyAllDeletes(deleteQueue);
|
||||
|
@ -500,7 +498,7 @@ final class DocumentsWriter {
|
|||
assert bufferedDeletes != null;
|
||||
if (bufferedDeletes != null && bufferedDeletes.any()) {
|
||||
indexWriter.publishFrozenDeletes(bufferedDeletes);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "flush: push buffered deletes: " + bufferedDeletes);
|
||||
}
|
||||
}
|
||||
|
@ -527,14 +525,14 @@ final class DocumentsWriter {
|
|||
assert newSegment != null;
|
||||
final SegmentInfo segInfo = indexWriter.prepareFlushedSegment(newSegment);
|
||||
final BufferedDeletes deletes = newSegment.segmentDeletes;
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": publishFlushedSegment seg-private deletes=" + deletes);
|
||||
}
|
||||
FrozenBufferedDeletes packet = null;
|
||||
if (deletes != null && deletes.any()) {
|
||||
// Segment private delete
|
||||
packet = new FrozenBufferedDeletes(deletes, true);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "flush: push buffered seg private deletes: " + packet);
|
||||
}
|
||||
}
|
||||
|
@ -560,7 +558,7 @@ final class DocumentsWriter {
|
|||
final boolean flushAllThreads()
|
||||
throws IOException {
|
||||
final DocumentsWriterDeleteQueue flushingDeleteQueue;
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + " startFullFlush");
|
||||
}
|
||||
|
||||
|
@ -586,7 +584,7 @@ final class DocumentsWriter {
|
|||
// If a concurrent flush is still in flight wait for it
|
||||
flushControl.waitForFlush();
|
||||
if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
|
||||
}
|
||||
synchronized (ticketQueue) {
|
||||
|
@ -604,7 +602,7 @@ final class DocumentsWriter {
|
|||
|
||||
final void finishFullFlush(boolean success) {
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + " finishFullFlush success=" + success);
|
||||
}
|
||||
assert setFlushingDeleteQueue(null);
|
||||
|
|
|
@ -453,7 +453,7 @@ public final class DocumentsWriterFlushControl {
|
|||
private final List<DocumentsWriterPerThread> fullFlushBuffer = new ArrayList<DocumentsWriterPerThread>();
|
||||
|
||||
void addFlushableState(ThreadState perThread) {
|
||||
if (documentsWriter.infoStream != null) {
|
||||
if (documentsWriter.infoStream.isEnabled("DWFC")) {
|
||||
documentsWriter.infoStream.message("DWFC", Thread.currentThread().getName() + ": addFlushableState " + perThread.perThread);
|
||||
}
|
||||
final DocumentsWriterPerThread dwpt = perThread.perThread;
|
||||
|
|
|
@ -131,9 +131,7 @@ public class DocumentsWriterPerThread {
|
|||
void abort() throws IOException {
|
||||
hasAborted = aborting = true;
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("DWPT", "now abort");
|
||||
}
|
||||
infoStream.message("DWPT", "now abort");
|
||||
try {
|
||||
consumer.abort();
|
||||
} catch (Throwable t) {
|
||||
|
@ -146,9 +144,7 @@ public class DocumentsWriterPerThread {
|
|||
|
||||
} finally {
|
||||
aborting = false;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("DWPT", "done abort");
|
||||
}
|
||||
infoStream.message("DWPT", "done abort");
|
||||
}
|
||||
}
|
||||
private final static boolean INFO_VERBOSE = false;
|
||||
|
@ -226,12 +222,12 @@ public class DocumentsWriterPerThread {
|
|||
// this call is synchronized on IndexWriter.segmentInfos
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
}
|
||||
|
||||
}
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
}
|
||||
boolean success = false;
|
||||
|
@ -274,11 +270,11 @@ public class DocumentsWriterPerThread {
|
|||
// this call is synchronized on IndexWriter.segmentInfos
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
}
|
||||
}
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
}
|
||||
int docCount = 0;
|
||||
|
@ -459,12 +455,12 @@ public class DocumentsWriterPerThread {
|
|||
pendingDeletes.docIDs.clear();
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
|
||||
}
|
||||
|
||||
if (aborting) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush: skip because aborting is set");
|
||||
}
|
||||
return null;
|
||||
|
@ -476,7 +472,7 @@ public class DocumentsWriterPerThread {
|
|||
consumer.flush(flushState);
|
||||
pendingDeletes.terms.clear();
|
||||
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
|
||||
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
|
||||
|
@ -493,7 +489,7 @@ public class DocumentsWriterPerThread {
|
|||
pendingDeletes = new BufferedDeletes();
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.;
|
||||
final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.;
|
||||
infoStream.message("DWPT", "flushed: segment=" + newSegment +
|
||||
|
|
|
@ -63,7 +63,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
if ((flushOnRAM() &&
|
||||
control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
|
||||
control.setApplyAllDeletes();
|
||||
if (writer.infoStream != null) {
|
||||
if (writer.infoStream.isEnabled("FP")) {
|
||||
writer.infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB()));
|
||||
}
|
||||
}
|
||||
|
@ -81,7 +81,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
final long totalRam = control.activeBytes() + control.getDeleteBytesUsed();
|
||||
if (totalRam >= limit) {
|
||||
final DocumentsWriter writer = this.writer.get();
|
||||
if (writer.infoStream != null) {
|
||||
if (writer.infoStream.isEnabled("FP")) {
|
||||
writer.infoStream.message("FP", "flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
|
||||
}
|
||||
markLargestWriterPending(control, state, totalRam);
|
||||
|
|
|
@ -129,10 +129,7 @@ public abstract class FlushPolicy {
|
|||
}
|
||||
|
||||
private boolean assertMessage(String s) {
|
||||
InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("FP", s);
|
||||
}
|
||||
writer.get().infoStream.message("FP", s);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -102,7 +102,7 @@ final class IndexFileDeleter {
|
|||
private SegmentInfos lastSegmentInfos;
|
||||
|
||||
/** Change to true to see details of reference counts when
|
||||
* infoStream != null */
|
||||
* infoStream is enabled */
|
||||
public static boolean VERBOSE_REF_COUNTS = false;
|
||||
|
||||
// Used only for assert
|
||||
|
@ -128,7 +128,7 @@ final class IndexFileDeleter {
|
|||
|
||||
final String currentSegmentsFile = segmentInfos.getCurrentSegmentFileName();
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: current segments file is \"" + currentSegmentsFile + "\"; deletionPolicy=" + policy);
|
||||
}
|
||||
|
||||
|
@ -160,7 +160,7 @@ final class IndexFileDeleter {
|
|||
// This is a commit (segments or segments_N), and
|
||||
// it's valid (<= the max gen). Load it, then
|
||||
// incref all files it refers to:
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
|
@ -174,7 +174,7 @@ final class IndexFileDeleter {
|
|||
// file segments_X exists when in fact it
|
||||
// doesn't. So, we catch this and handle it
|
||||
// as if the file does not exist
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
}
|
||||
sis = null;
|
||||
|
@ -205,7 +205,7 @@ final class IndexFileDeleter {
|
|||
} catch (FileNotFoundException e) {
|
||||
refresh(segmentInfo.name);
|
||||
sis = null;
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
}
|
||||
}
|
||||
|
@ -242,7 +242,7 @@ final class IndexFileDeleter {
|
|||
} catch (IOException e) {
|
||||
throw new CorruptIndexException("failed to locate current segments_N file");
|
||||
}
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
|
||||
}
|
||||
currentCommitPoint = new CommitPoint(commitsToDelete, directory, sis);
|
||||
|
@ -260,7 +260,7 @@ final class IndexFileDeleter {
|
|||
RefCount rc = entry.getValue();
|
||||
final String fileName = entry.getKey();
|
||||
if (0 == rc.count) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
|
@ -300,7 +300,7 @@ final class IndexFileDeleter {
|
|||
// the now-deleted commits:
|
||||
for(int i=0;i<size;i++) {
|
||||
CommitPoint commit = commitsToDelete.get(i);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
for (final String file : commit.files) {
|
||||
|
@ -360,7 +360,7 @@ final class IndexFileDeleter {
|
|||
!refCounts.containsKey(fileName) &&
|
||||
!fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
|
||||
// Unreferenced file, so remove it
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
|
@ -402,7 +402,7 @@ final class IndexFileDeleter {
|
|||
*/
|
||||
void revisitPolicy() throws IOException {
|
||||
assert locked();
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "now revisitPolicy");
|
||||
}
|
||||
|
||||
|
@ -419,7 +419,7 @@ final class IndexFileDeleter {
|
|||
deletable = null;
|
||||
int size = oldDeletable.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete pending file " + oldDeletable.get(i));
|
||||
}
|
||||
deleteFile(oldDeletable.get(i));
|
||||
|
@ -450,7 +450,7 @@ final class IndexFileDeleter {
|
|||
public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
|
||||
assert locked();
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "now checkpoint \"" + segmentInfos.toString(directory) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
|
@ -501,7 +501,7 @@ final class IndexFileDeleter {
|
|||
void incRef(String fileName) throws IOException {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream != null && VERBOSE_REF_COUNTS) {
|
||||
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
|
||||
}
|
||||
rc.IncRef();
|
||||
|
@ -517,7 +517,7 @@ final class IndexFileDeleter {
|
|||
void decRef(String fileName) throws IOException {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream != null && VERBOSE_REF_COUNTS) {
|
||||
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
|
||||
}
|
||||
if (0 == rc.DecRef()) {
|
||||
|
@ -569,7 +569,7 @@ final class IndexFileDeleter {
|
|||
assert locked();
|
||||
for (final String fileName: files) {
|
||||
if (!refCounts.containsKey(fileName)) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete new file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
|
@ -581,7 +581,7 @@ final class IndexFileDeleter {
|
|||
throws IOException {
|
||||
assert locked();
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete \"" + fileName + "\"");
|
||||
}
|
||||
directory.deleteFile(fileName);
|
||||
|
@ -595,7 +595,7 @@ final class IndexFileDeleter {
|
|||
// the file is open in another process, and queue
|
||||
// the file for subsequent deletion.
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
}
|
||||
if (deletable == null) {
|
||||
|
|
|
@ -131,11 +131,11 @@ public final class IndexUpgrader {
|
|||
final IndexWriter w = new IndexWriter(dir, c);
|
||||
try {
|
||||
InfoStream infoStream = c.getInfoStream();
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IndexUpgrader")) {
|
||||
infoStream.message("IndexUpgrader", "Upgrading all pre-" + Constants.LUCENE_MAIN_VERSION + " segments of index directory '" + dir + "' to version " + Constants.LUCENE_MAIN_VERSION + "...");
|
||||
}
|
||||
w.forceMerge(1);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IndexUpgrader")) {
|
||||
infoStream.message("IndexUpgrader", "All segments upgraded to version " + Constants.LUCENE_MAIN_VERSION);
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -331,9 +331,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final long tStart = System.currentTimeMillis();
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "flush at getReader");
|
||||
}
|
||||
infoStream.message("IW", "flush at getReader");
|
||||
// Do this up front before flushing so that the readers
|
||||
// obtained during this flush are pooled, the first time
|
||||
// this method is called:
|
||||
|
@ -364,7 +362,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
synchronized(this) {
|
||||
maybeApplyDeletes(applyAllDeletes);
|
||||
r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
|
||||
}
|
||||
}
|
||||
|
@ -373,9 +371,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// never reached but javac disagrees:
|
||||
return null;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
}
|
||||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(success);
|
||||
doAfterFlush();
|
||||
|
@ -384,7 +380,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (anySegmentFlushed) {
|
||||
maybeMerge();
|
||||
}
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
|
||||
}
|
||||
return r;
|
||||
|
@ -874,7 +870,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segmentInfos.replace(oldInfos);
|
||||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
}
|
||||
|
@ -902,7 +898,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segmentInfos.changed();
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "init: create=" + create);
|
||||
messageState();
|
||||
}
|
||||
|
@ -911,9 +907,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "init: hit exception on init; releasing write lock");
|
||||
}
|
||||
infoStream.message("IW", "init: hit exception on init; releasing write lock");
|
||||
try {
|
||||
writeLock.release();
|
||||
} catch (Throwable t) {
|
||||
|
@ -941,7 +935,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
private void messageState() throws IOException {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "\ndir=" + directory + "\n" +
|
||||
"index=" + segString() + "\n" +
|
||||
"version=" + Constants.LUCENE_VERSION + "\n" +
|
||||
|
@ -949,11 +943,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
}
|
||||
|
||||
/** Returns true if verbosing is enabled (i.e., infoStream != null). */
|
||||
public boolean verbose() {
|
||||
return infoStream != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits all changes to an index and closes all
|
||||
* associated files. Note that this may be a costly
|
||||
|
@ -1060,7 +1049,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
private void closeInternal(boolean waitForMerges) throws CorruptIndexException, IOException {
|
||||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now flush at close waitForMerges=" + waitForMerges);
|
||||
}
|
||||
|
||||
|
@ -1086,14 +1075,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
mergeScheduler.close();
|
||||
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "now call final commit()");
|
||||
infoStream.message("IW", "now call final commit()");
|
||||
|
||||
if (!hitOOM) {
|
||||
commitInternal(null);
|
||||
}
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "at close: " + segString());
|
||||
// used by assert below
|
||||
final DocumentsWriter oldWriter = docWriter;
|
||||
|
@ -1118,8 +1106,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
closing = false;
|
||||
notifyAll();
|
||||
if (!closed) {
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "hit exception while closing");
|
||||
infoStream.message("IW", "hit exception while closing");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1348,9 +1335,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
anySegmentFlushed = docWriter.updateDocuments(docs, analyzer, delTerm);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
if (anySegmentFlushed) {
|
||||
maybeMerge();
|
||||
|
@ -1495,8 +1480,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
anySegmentFlushed = docWriter.updateDocument(doc, analyzer, term);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null)
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
|
||||
if (anySegmentFlushed) {
|
||||
|
@ -1644,7 +1628,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (maxNumSegments < 1)
|
||||
throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "forceMerge: index now " + segString());
|
||||
infoStream.message("IW", "now flush at forceMerge");
|
||||
}
|
||||
|
@ -1753,7 +1737,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
flush(true, true);
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "forceMergeDeletes: index now " + segString());
|
||||
|
||||
MergePolicy.MergeSpecification spec;
|
||||
|
@ -1947,9 +1931,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
boolean success = false;
|
||||
|
||||
if (infoStream != null ) {
|
||||
infoStream.message("IW", "rollback");
|
||||
}
|
||||
infoStream.message("IW", "rollback");
|
||||
|
||||
try {
|
||||
synchronized(this) {
|
||||
|
@ -1957,9 +1939,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
stopMerges = true;
|
||||
}
|
||||
|
||||
if (infoStream != null ) {
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
}
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
|
||||
// Must pre-close these two, in case they increment
|
||||
// changeCount so that we can then set it to false
|
||||
|
@ -1984,7 +1964,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// will always write to a new generation ("write
|
||||
// once").
|
||||
segmentInfos.rollbackSegmentInfos(rollbackSegments);
|
||||
if (infoStream != null ) {
|
||||
if (infoStream.isEnabled("IW") ) {
|
||||
infoStream.message("IW", "rollback: infos=" + segString(segmentInfos));
|
||||
}
|
||||
|
||||
|
@ -2011,8 +1991,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (!success) {
|
||||
closing = false;
|
||||
notifyAll();
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "hit exception during rollback");
|
||||
infoStream.message("IW", "hit exception during rollback");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2064,9 +2043,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteAll");
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
infoStream.message("IW", "hit exception during deleteAll");
|
||||
}
|
||||
infoStream.message("IW", "hit exception during deleteAll");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2077,7 +2054,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// Abort all pending & running merges:
|
||||
for (final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "now abort pending merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
mergeFinish(merge);
|
||||
|
@ -2085,7 +2062,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
pendingMerges.clear();
|
||||
|
||||
for (final MergePolicy.OneMerge merge : runningMerges) {
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "now abort running merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
}
|
||||
|
@ -2096,7 +2073,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// because the merge threads periodically check if
|
||||
// they are aborted.
|
||||
while(runningMerges.size() > 0) {
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge to abort");
|
||||
doWait();
|
||||
}
|
||||
|
@ -2106,8 +2083,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
assert 0 == mergingSegments.size();
|
||||
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
|
||||
} else {
|
||||
// waitForMerges() will ensure any running addIndexes finishes.
|
||||
|
@ -2127,9 +2103,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
*/
|
||||
public synchronized void waitForMerges() {
|
||||
ensureOpen(false);
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
}
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
||||
doWait();
|
||||
}
|
||||
|
@ -2137,9 +2111,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// sanity check
|
||||
assert 0 == mergingSegments.size();
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "waitForMerges done");
|
||||
}
|
||||
infoStream.message("IW", "waitForMerges done");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2175,7 +2147,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
if (useCompoundFile(newSegment)) {
|
||||
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "creating compound file " + compoundFileName);
|
||||
}
|
||||
// Now build compound file
|
||||
|
@ -2207,7 +2179,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
newSegment.setDelCount(delCount);
|
||||
newSegment.advanceDelGen();
|
||||
final String delFileName = newSegment.getDelFileName();
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush: write " + delCount + " deletes to " + delFileName);
|
||||
}
|
||||
boolean success2 = false;
|
||||
|
@ -2234,7 +2206,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception " +
|
||||
"reating compound file for newly flushed segment " + newSegment.name);
|
||||
}
|
||||
|
@ -2266,9 +2238,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException {
|
||||
// Lock order IW -> BDS
|
||||
synchronized (bufferedDeletesStream) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
}
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
|
||||
if (globalPacket != null && globalPacket.any()) {
|
||||
bufferedDeletesStream.push(globalPacket);
|
||||
|
@ -2283,7 +2253,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// generation right away
|
||||
nextGen = bufferedDeletesStream.getNextGen();
|
||||
}
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "publish sets newSegment delGen=" + nextGen);
|
||||
}
|
||||
newSegment.setBufferedDeletesGen(nextGen);
|
||||
|
@ -2362,13 +2332,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
noDupDirs(dirs);
|
||||
|
||||
try {
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "flush at addIndexes(Directory...)");
|
||||
infoStream.message("IW", "flush at addIndexes(Directory...)");
|
||||
flush(false, true);
|
||||
|
||||
List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
|
||||
for (Directory dir : dirs) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "addIndexes: process directory " + dir);
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos(); // read infos from dir
|
||||
|
@ -2381,7 +2350,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
String newSegName = newSegmentName();
|
||||
String dsName = info.getDocStoreSegment();
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
|
||||
}
|
||||
|
||||
|
@ -2432,8 +2401,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
int numDocs = 0;
|
||||
|
||||
try {
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
|
||||
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
|
||||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
|
@ -2600,7 +2568,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
ensureOpen(false);
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "prepareCommit: flush");
|
||||
infoStream.message("IW", " index before flush " + segString());
|
||||
}
|
||||
|
@ -2660,7 +2628,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
if (!success) {
|
||||
infoStream.message("IW", "hit exception during prepareCommit");
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
|
@ -2745,21 +2713,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "commit: start");
|
||||
}
|
||||
infoStream.message("IW", "commit: start");
|
||||
|
||||
synchronized(commitLock) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "commit: enter lock");
|
||||
}
|
||||
infoStream.message("IW", "commit: enter lock");
|
||||
|
||||
if (pendingCommit == null) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "commit: now prepare");
|
||||
}
|
||||
infoStream.message("IW", "commit: now prepare");
|
||||
prepareCommit(commitUserData);
|
||||
} else if (infoStream != null) {
|
||||
} else {
|
||||
infoStream.message("IW", "commit: already prepared");
|
||||
}
|
||||
|
||||
|
@ -2771,10 +2733,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (pendingCommit != null) {
|
||||
try {
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
pendingCommit.finishCommit(directory, codec);
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
|
@ -2789,13 +2750,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
notifyAll();
|
||||
}
|
||||
|
||||
} else if (infoStream != null) {
|
||||
} else {
|
||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "commit: done");
|
||||
}
|
||||
infoStream.message("IW", "commit: done");
|
||||
}
|
||||
|
||||
// Ensures only one flush() is actually flushing segments
|
||||
|
@ -2835,7 +2794,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
boolean success = false;
|
||||
try {
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", " start flush: applyAllDeletes=" + applyAllDeletes);
|
||||
infoStream.message("IW", " index before flush " + segString());
|
||||
}
|
||||
|
@ -2865,18 +2824,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// never hit
|
||||
return false;
|
||||
} finally {
|
||||
if (!success && infoStream != null)
|
||||
if (!success)
|
||||
infoStream.message("IW", "hit exception during flush");
|
||||
}
|
||||
}
|
||||
|
||||
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
|
||||
if (applyAllDeletes) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "apply all deletes during flush");
|
||||
}
|
||||
infoStream.message("IW", "apply all deletes during flush");
|
||||
applyAllDeletes();
|
||||
} else if (infoStream != null) {
|
||||
} else if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
|
||||
}
|
||||
}
|
||||
|
@ -2889,7 +2846,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
checkpoint();
|
||||
}
|
||||
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + segString(result.allDeleted));
|
||||
}
|
||||
for (SegmentInfo info : result.allDeleted) {
|
||||
|
@ -2954,7 +2911,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final List<SegmentInfo> sourceSegments = merge.segments;
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory));
|
||||
|
||||
// Carefully merge deletes that occurred after we
|
||||
|
@ -3052,7 +3009,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
|
||||
}
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "commitMerge: " + merge.segString(directory) + " index=" + segString());
|
||||
|
||||
assert merge.registerDone;
|
||||
|
@ -3064,7 +3021,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// file that current segments does not reference), we
|
||||
// abort this merge
|
||||
if (merge.isAborted()) {
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
|
||||
return false;
|
||||
}
|
||||
|
@ -3080,7 +3037,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final boolean allDeleted = mergedReader.numDocs() == 0;
|
||||
|
||||
if (infoStream != null && allDeleted) {
|
||||
if (allDeleted && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
|
||||
}
|
||||
|
||||
|
@ -3091,7 +3048,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
readerPool.drop(merge.info);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "after commit: " + segString());
|
||||
}
|
||||
|
||||
|
@ -3118,7 +3075,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final private void handleMergeException(Throwable t, MergePolicy.OneMerge merge) throws IOException {
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "handleMergeException: merge=" + merge.segString(directory) + " exc=" + t);
|
||||
}
|
||||
|
||||
|
@ -3167,7 +3124,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
mergeInit(merge);
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "now merge\n merge=" + merge.segString(directory) + "\n index=" + segString());
|
||||
|
||||
mergeMiddle(merge);
|
||||
|
@ -3181,8 +3138,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergeFinish(merge);
|
||||
|
||||
if (!success) {
|
||||
if (infoStream != null)
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info))
|
||||
deleter.refresh(merge.info.name);
|
||||
}
|
||||
|
@ -3198,7 +3154,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "merge");
|
||||
}
|
||||
if (infoStream != null && merge.info != null) {
|
||||
if (merge.info != null && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": merge end");
|
||||
|
@ -3244,7 +3200,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
pendingMerges.add(merge);
|
||||
|
||||
if (infoStream != null)
|
||||
if (infoStream.isEnabled("IW"))
|
||||
infoStream.message("IW", "add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
|
||||
|
||||
merge.mergeGen = mergeGen;
|
||||
|
@ -3254,7 +3210,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// is running (while synchronized) to avoid race
|
||||
// condition where two conflicting merges from different
|
||||
// threads, start
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
StringBuilder builder = new StringBuilder("registerMerge merging= [");
|
||||
for (SegmentInfo info : mergingSegments) {
|
||||
builder.append(info.name).append(", ");
|
||||
|
@ -3265,7 +3221,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
infoStream.message("IW", builder.toString());
|
||||
}
|
||||
for(SegmentInfo info : merge.segments) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "registerMerge info=" + info);
|
||||
}
|
||||
mergingSegments.add(info);
|
||||
|
@ -3286,9 +3242,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "hit exception in mergeInit");
|
||||
}
|
||||
infoStream.message("IW", "hit exception in mergeInit");
|
||||
mergeFinish(merge);
|
||||
}
|
||||
}
|
||||
|
@ -3328,7 +3282,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + result.allDeleted);
|
||||
}
|
||||
for(SegmentInfo info : result.allDeleted) {
|
||||
|
@ -3353,7 +3307,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
details.put("mergeFactor", Integer.toString(merge.segments.size()));
|
||||
setDiagnostics(merge.info, "merge", details);
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge seg=" + merge.info.name);
|
||||
}
|
||||
|
||||
|
@ -3495,7 +3449,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort,
|
||||
payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
|
||||
}
|
||||
|
||||
|
@ -3529,7 +3483,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segUpto++;
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge: total " + totDocCount + " docs");
|
||||
}
|
||||
|
||||
|
@ -3542,7 +3496,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Record which codec was used to write the segment
|
||||
merge.info.setCodec(codec);
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge codec=" + codec);
|
||||
}
|
||||
|
||||
|
@ -3562,7 +3516,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final String compoundFileName = IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "create compound file " + compoundFileName);
|
||||
}
|
||||
createCompoundFile(directory, compoundFileName, checkAbort, merge.info, new IOContext(merge.getMergeInfo()));
|
||||
|
@ -3581,9 +3535,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
handleMergeException(t, merge);
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
}
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
|
||||
synchronized(this) {
|
||||
deleter.deleteFile(compoundFileName);
|
||||
|
@ -3602,9 +3554,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
deleter.deleteNewFiles(merge.info.files());
|
||||
|
||||
if (merge.isAborted()) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
}
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
deleter.deleteFile(compoundFileName);
|
||||
return 0;
|
||||
}
|
||||
|
@ -3613,7 +3563,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
merge.info.setUseCompoundFile(true);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes()/1024./1024., merge.estimatedMergeBytes/1024/1024.));
|
||||
}
|
||||
|
||||
|
@ -3787,24 +3737,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "startCommit(): start");
|
||||
}
|
||||
infoStream.message("IW", "startCommit(): start");
|
||||
|
||||
synchronized(this) {
|
||||
|
||||
assert lastCommitChangeCount <= changeCount;
|
||||
|
||||
if (pendingCommitChangeCount == lastCommitChangeCount) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
}
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
return;
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
|
||||
}
|
||||
|
||||
|
@ -3841,9 +3787,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
pendingCommit = toSync;
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "done all syncs");
|
||||
}
|
||||
infoStream.message("IW", "done all syncs");
|
||||
|
||||
assert testPoint("midStartCommitSuccess");
|
||||
|
||||
|
@ -3856,9 +3800,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segmentInfos.updateGeneration(toSync);
|
||||
|
||||
if (!pendingCommitSet) {
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "hit exception committing segments file");
|
||||
}
|
||||
infoStream.message("IW", "hit exception committing segments file");
|
||||
|
||||
// Hit exception
|
||||
deleter.decRef(filesToCommit);
|
||||
|
@ -3910,7 +3852,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
private void handleOOM(OutOfMemoryError oom, String location) {
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit OutOfMemoryError inside " + location);
|
||||
}
|
||||
hitOOM = true;
|
||||
|
@ -3935,7 +3877,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
synchronized boolean nrtIsCurrent(SegmentInfos infos) {
|
||||
//System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
|
||||
ensureOpen();
|
||||
if (infoStream != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + " DW changes: " + docWriter.anyChanges() + " BD changes: "+bufferedDeletesStream.any());
|
||||
|
||||
}
|
||||
|
|
|
@ -695,6 +695,10 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
* to this.
|
||||
*/
|
||||
public IndexWriterConfig setInfoStream(InfoStream infoStream) {
|
||||
if (infoStream == null) {
|
||||
throw new IllegalArgumentException("Cannot set InfoStream implementation to null. "+
|
||||
"To disable logging use InfoStream.NO_OUTPUT");
|
||||
}
|
||||
this.infoStream = infoStream;
|
||||
return this;
|
||||
}
|
||||
|
@ -703,8 +707,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
* Convenience method that uses {@link PrintStreamInfoStream}
|
||||
*/
|
||||
public IndexWriterConfig setInfoStream(PrintStream printStream) {
|
||||
this.infoStream = printStream == null ? null : new PrintStreamInfoStream(printStream);
|
||||
return this;
|
||||
return setInfoStream(printStream == null ? InfoStream.NO_OUTPUT : new PrintStreamInfoStream(printStream));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -725,7 +728,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
sb.append("maxBufferedDocs=").append(maxBufferedDocs).append("\n");
|
||||
sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n");
|
||||
sb.append("codec=").append(codec).append("\n");
|
||||
sb.append("infoStream=").append(infoStream == null ? "null" : infoStream.getClass().getName()).append("\n");
|
||||
sb.append("infoStream=").append(infoStream.getClass().getName()).append("\n");
|
||||
sb.append("mergePolicy=").append(mergePolicy).append("\n");
|
||||
sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n");
|
||||
sb.append("readerPooling=").append(readerPooling).append("\n");
|
||||
|
|
|
@ -84,8 +84,8 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
protected boolean verbose() {
|
||||
IndexWriter w = writer.get();
|
||||
return w != null && w.verbose();
|
||||
final IndexWriter w = writer.get();
|
||||
return w != null && w.infoStream.isEnabled("LMP");
|
||||
}
|
||||
|
||||
/** @see #setNoCFSRatio */
|
||||
|
@ -107,10 +107,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
|
||||
protected void message(String message) {
|
||||
if (verbose()) {
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("LMP", message);
|
||||
}
|
||||
writer.get().infoStream.message("LMP", message);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -175,7 +175,7 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
if (mergeState.infoStream != null) {
|
||||
if (mergeState.infoStream.isEnabled("SM")) {
|
||||
mergeState.infoStream.message("SM", "merge store matchedCount=" + mergeState.matchedCount + " vs " + mergeState.readers.size());
|
||||
if (mergeState.matchedCount != mergeState.readers.size()) {
|
||||
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - mergeState.matchedCount) + " non-bulk merges");
|
||||
|
|
|
@ -644,16 +644,13 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
private boolean verbose() {
|
||||
IndexWriter w = writer.get();
|
||||
return w != null && w.verbose();
|
||||
final IndexWriter w = writer.get();
|
||||
return w != null && w.infoStream.isEnabled("TMP");
|
||||
}
|
||||
|
||||
private void message(String message) {
|
||||
if (verbose()) {
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("TMP", message);
|
||||
}
|
||||
writer.get().infoStream.message("TMP", message);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -147,16 +147,13 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
private boolean verbose() {
|
||||
IndexWriter w = writer.get();
|
||||
return w != null && w.verbose();
|
||||
final IndexWriter w = writer.get();
|
||||
return w != null && w.infoStream.isEnabled("UPGMP");
|
||||
}
|
||||
|
||||
private void message(String message) {
|
||||
if (verbose()) {
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("UPGMP", message);
|
||||
}
|
||||
writer.get().infoStream.message("UPGMP", message);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,30 +18,47 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/** @lucene.internal */
|
||||
public abstract class InfoStream implements Closeable {
|
||||
// Used for printing messages
|
||||
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
|
||||
protected final int messageID = MESSAGE_ID.getAndIncrement();
|
||||
|
||||
/** Instance of InfoStream that does no logging at all. */
|
||||
public static final InfoStream NO_OUTPUT = new NoOutput();
|
||||
private static final class NoOutput extends InfoStream {
|
||||
@Override
|
||||
public void message(String component, String message) {}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) { return false; }
|
||||
|
||||
@Override
|
||||
public void close() {}
|
||||
}
|
||||
|
||||
/** prints a message */
|
||||
public abstract void message(String component, String message);
|
||||
|
||||
private static InfoStream defaultInfoStream;
|
||||
/** returns true if messages are enabled and should be posted to {@link #message}. */
|
||||
public abstract boolean isEnabled(String component);
|
||||
|
||||
/** The default infoStream (possibly null) used
|
||||
* by a newly instantiated classes.
|
||||
private static InfoStream defaultInfoStream = NO_OUTPUT;
|
||||
|
||||
/** The default {@code InfoStream} used by a newly instantiated classes.
|
||||
* @see #setDefault */
|
||||
public static InfoStream getDefault() {
|
||||
public static synchronized InfoStream getDefault() {
|
||||
return defaultInfoStream;
|
||||
}
|
||||
|
||||
/** Sets the default infoStream (possibly null) used
|
||||
* by a newly instantiated classes.
|
||||
* @see #setDefault */
|
||||
public static void setDefault(InfoStream infoStream) {
|
||||
/** Sets the default {@code InfoStream} used
|
||||
* by a newly instantiated classes. It cannot be {@code null},
|
||||
* to disable logging use {@link #NO_OUTPUT}.
|
||||
* @see #getDefault */
|
||||
public static synchronized void setDefault(InfoStream infoStream) {
|
||||
if (infoStream == null) {
|
||||
throw new IllegalArgumentException("Cannot set InfoStream default implementation to null. "+
|
||||
"To disable logging use InfoStream.NO_OUTPUT");
|
||||
}
|
||||
defaultInfoStream = infoStream;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,15 +20,25 @@ package org.apache.lucene.util;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Date;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class PrintStreamInfoStream extends InfoStream {
|
||||
// Used for printing messages
|
||||
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
|
||||
protected final int messageID;
|
||||
|
||||
private final PrintStream stream;
|
||||
|
||||
public PrintStreamInfoStream(PrintStream stream) {
|
||||
this(stream, MESSAGE_ID.getAndIncrement());
|
||||
}
|
||||
|
||||
public PrintStreamInfoStream(PrintStream stream, int messageID) {
|
||||
this.stream = stream;
|
||||
this.messageID = messageID;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -36,6 +46,11 @@ public class PrintStreamInfoStream extends InfoStream {
|
|||
stream.println(component + " " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (!isSystemStream()) {
|
||||
|
|
|
@ -27,6 +27,11 @@ public class FailOnNonBulkMergesInfoStream extends InfoStream {
|
|||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void message(String component, String message) {
|
||||
assert !message.contains("non-bulk merges");
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.util;
|
|||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Prints nothing. Just to make sure tests pass w/ and without infostream
|
||||
* Prints nothing. Just to make sure tests pass w/ and without enabled InfoStream
|
||||
* without actually making noise.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -31,6 +31,12 @@ public class NullInfoStream extends InfoStream {
|
|||
assert message != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) {
|
||||
assert component != null;
|
||||
return true; // to actually enable logging, we just ignore on message()
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.ArrayList;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.document.Document;
|
||||
|
@ -958,18 +959,22 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
// LUCENE-1429
|
||||
public void testOutOfMemoryErrorCausesCloseToFail() throws Exception {
|
||||
|
||||
final List<Throwable> thrown = new ArrayList<Throwable>();
|
||||
final AtomicBoolean thrown = new AtomicBoolean(false);
|
||||
final Directory dir = newDirectory();
|
||||
final IndexWriter writer = new IndexWriter(dir,
|
||||
newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new InfoStream() {
|
||||
@Override
|
||||
public void message(String component, final String message) {
|
||||
if (message.startsWith("now flush at close") && 0 == thrown.size()) {
|
||||
thrown.add(null);
|
||||
if (message.startsWith("now flush at close") && thrown.compareAndSet(false, true)) {
|
||||
throw new OutOfMemoryError("fake OOME at " + message);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {}
|
||||
}));
|
||||
|
|
|
@ -74,9 +74,9 @@ public class SolrIndexWriter extends IndexWriter {
|
|||
File parent = f.getParentFile();
|
||||
if (parent != null) parent.mkdirs();
|
||||
FileOutputStream fos = new FileOutputStream(f, true);
|
||||
return new PrintStreamInfoStream(new TimeLoggingPrintStream(fos, true));
|
||||
return new PrintStreamInfoStream(new PrintStream(fos, true));
|
||||
} else {
|
||||
return null;
|
||||
return InfoStream.NO_OUTPUT;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -155,23 +155,4 @@ public class SolrIndexWriter extends IndexWriter {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
// Helper class for adding timestamps to infoStream logging
|
||||
static class TimeLoggingPrintStream extends PrintStream {
|
||||
private DateFormat dateFormat;
|
||||
public TimeLoggingPrintStream(OutputStream underlyingOutputStream,
|
||||
boolean autoFlush) {
|
||||
super(underlyingOutputStream, autoFlush);
|
||||
this.dateFormat = DateFormat.getDateTimeInstance();
|
||||
}
|
||||
|
||||
// We might ideally want to override print(String) as well, but
|
||||
// looking through the code that writes to infoStream, it appears
|
||||
// that all the classes except CheckIndex just use println.
|
||||
@Override
|
||||
public void println(String x) {
|
||||
print(dateFormat.format(new Date()) + " ");
|
||||
super.println(x);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue