mirror of
https://github.com/apache/lucene.git
synced 2025-02-09 11:35:14 +00:00
LUCENE-5756: Implement Accountable from IndexWriter
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1602388 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6f7949c940
commit
d8fb251c05
@ -156,6 +156,9 @@ Changes in Backwards Compatibility Policy
|
|||||||
|
|
||||||
API Changes
|
API Changes
|
||||||
|
|
||||||
|
* LUCENE-5756: IndexWriter now implements Accountable and IW#ramSizeInBytes()
|
||||||
|
has been deprecated infavor of IW#ramBytesUsed() (Simon Willnauer)
|
||||||
|
|
||||||
* LUCENE-5725: MoreLikeThis#like now accepts multiple values per field.
|
* LUCENE-5725: MoreLikeThis#like now accepts multiple values per field.
|
||||||
The pre-existing method has been deprecated in favor of a variable arguments
|
The pre-existing method has been deprecated in favor of a variable arguments
|
||||||
for the like text. (Alex Ksikes via Simon Willnauer)
|
for the like text. (Alex Ksikes via Simon Willnauer)
|
||||||
|
@ -32,6 +32,7 @@ import org.apache.lucene.search.Query;
|
|||||||
import org.apache.lucene.search.QueryWrapperFilter;
|
import org.apache.lucene.search.QueryWrapperFilter;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.IOContext;
|
import org.apache.lucene.store.IOContext;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
|
|
||||||
@ -50,7 +51,7 @@ import org.apache.lucene.util.InfoStream;
|
|||||||
* track which BufferedDeletes packets to apply to any given
|
* track which BufferedDeletes packets to apply to any given
|
||||||
* segment. */
|
* segment. */
|
||||||
|
|
||||||
class BufferedUpdatesStream {
|
class BufferedUpdatesStream implements Accountable {
|
||||||
|
|
||||||
// TODO: maybe linked list?
|
// TODO: maybe linked list?
|
||||||
private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
|
private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
|
||||||
@ -111,7 +112,8 @@ class BufferedUpdatesStream {
|
|||||||
return numTerms.get();
|
return numTerms.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long bytesUsed() {
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
return bytesUsed.get();
|
return bytesUsed.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexWriter.Event;
|
|||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -94,7 +95,7 @@ import org.apache.lucene.util.InfoStream;
|
|||||||
* or none") added to the index.
|
* or none") added to the index.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
final class DocumentsWriter implements Closeable {
|
final class DocumentsWriter implements Closeable, Accountable {
|
||||||
private final Directory directory;
|
private final Directory directory;
|
||||||
|
|
||||||
private volatile boolean closed;
|
private volatile boolean closed;
|
||||||
@ -663,6 +664,11 @@ final class DocumentsWriter implements Closeable {
|
|||||||
events.add(event);
|
events.add(event);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return flushControl.ramBytesUsed();
|
||||||
|
}
|
||||||
|
|
||||||
static final class ApplyDeletesEvent implements Event {
|
static final class ApplyDeletesEvent implements Event {
|
||||||
static final Event INSTANCE = new ApplyDeletesEvent();
|
static final Event INSTANCE = new ApplyDeletesEvent();
|
||||||
private int instCount = 0;
|
private int instCount = 0;
|
||||||
|
@ -24,6 +24,7 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -66,7 +67,7 @@ import org.apache.lucene.util.BytesRef;
|
|||||||
* will also not be added to its private deletes neither to the global deletes.
|
* will also not be added to its private deletes neither to the global deletes.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
final class DocumentsWriterDeleteQueue {
|
final class DocumentsWriterDeleteQueue implements Accountable {
|
||||||
|
|
||||||
private volatile Node<?> tail;
|
private volatile Node<?> tail;
|
||||||
|
|
||||||
@ -449,7 +450,8 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public long bytesUsed() {
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
return globalBufferedUpdates.bytesUsed.get();
|
return globalBufferedUpdates.bytesUsed.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -19,14 +19,15 @@ package org.apache.lucene.index;
|
|||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.IdentityHashMap;
|
import java.util.IdentityHashMap;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Queue;
|
import java.util.Queue;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
import org.apache.lucene.util.ThreadInterruptedException;
|
import org.apache.lucene.util.ThreadInterruptedException;
|
||||||
|
|
||||||
@ -42,7 +43,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
|||||||
* {@link IndexWriterConfig#getRAMPerThreadHardLimitMB()} to prevent address
|
* {@link IndexWriterConfig#getRAMPerThreadHardLimitMB()} to prevent address
|
||||||
* space exhaustion.
|
* space exhaustion.
|
||||||
*/
|
*/
|
||||||
final class DocumentsWriterFlushControl {
|
final class DocumentsWriterFlushControl implements Accountable {
|
||||||
|
|
||||||
private final long hardMaxBytesPerDWPT;
|
private final long hardMaxBytesPerDWPT;
|
||||||
private long activeBytes = 0;
|
private long activeBytes = 0;
|
||||||
@ -428,7 +429,12 @@ final class DocumentsWriterFlushControl {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public long getDeleteBytesUsed() {
|
public long getDeleteBytesUsed() {
|
||||||
return documentsWriter.deleteQueue.bytesUsed() + bufferedUpdatesStream.bytesUsed();
|
return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return getDeleteBytesUsed() + netBytes();
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized int numFlushingDWPT() {
|
synchronized int numFlushingDWPT() {
|
||||||
|
@ -56,6 +56,7 @@ import org.apache.lucene.store.Lock;
|
|||||||
import org.apache.lucene.store.LockObtainFailedException;
|
import org.apache.lucene.store.LockObtainFailedException;
|
||||||
import org.apache.lucene.store.MergeInfo;
|
import org.apache.lucene.store.MergeInfo;
|
||||||
import org.apache.lucene.store.TrackingDirectoryWrapper;
|
import org.apache.lucene.store.TrackingDirectoryWrapper;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.Constants;
|
import org.apache.lucene.util.Constants;
|
||||||
@ -192,7 +193,7 @@ import org.apache.lucene.util.Version;
|
|||||||
* referenced by the "front" of the index). For this, IndexFileDeleter
|
* referenced by the "front" of the index). For this, IndexFileDeleter
|
||||||
* keeps track of the last non commit checkpoint.
|
* keeps track of the last non commit checkpoint.
|
||||||
*/
|
*/
|
||||||
public class IndexWriter implements Closeable, TwoPhaseCommit{
|
public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
|
private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
|
||||||
|
|
||||||
@ -425,6 +426,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
|||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final long ramBytesUsed() {
|
||||||
|
ensureOpen();
|
||||||
|
return docWriter.ramBytesUsed();
|
||||||
|
}
|
||||||
|
|
||||||
/** Holds shared SegmentReader instances. IndexWriter uses
|
/** Holds shared SegmentReader instances. IndexWriter uses
|
||||||
* SegmentReaders for 1) applying deletes, 2) doing
|
* SegmentReaders for 1) applying deletes, 2) doing
|
||||||
* merges, 3) handing out a real-time reader. This pool
|
* merges, 3) handing out a real-time reader. This pool
|
||||||
@ -3090,7 +3097,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
|||||||
}
|
}
|
||||||
applyAllDeletesAndUpdates();
|
applyAllDeletesAndUpdates();
|
||||||
} else if (infoStream.isEnabled("IW")) {
|
} else if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.bytesUsed());
|
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.ramBytesUsed());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -3120,14 +3127,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
|||||||
bufferedUpdatesStream.prune(segmentInfos);
|
bufferedUpdatesStream.prune(segmentInfos);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Expert: Return the total size of all index files currently cached in memory.
|
|
||||||
* Useful for size management with flushRamDocs()
|
|
||||||
*/
|
|
||||||
public final long ramSizeInBytes() {
|
|
||||||
ensureOpen();
|
|
||||||
return docWriter.flushControl.netBytes() + bufferedUpdatesStream.bytesUsed();
|
|
||||||
}
|
|
||||||
|
|
||||||
// for testing only
|
// for testing only
|
||||||
DocumentsWriter getDocsWriter() {
|
DocumentsWriter getDocsWriter() {
|
||||||
boolean test = false;
|
boolean test = false;
|
||||||
|
@ -316,7 +316,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
|
|||||||
while (pendingDocs.decrementAndGet() > -1) {
|
while (pendingDocs.decrementAndGet() > -1) {
|
||||||
Document doc = docs.nextDoc();
|
Document doc = docs.nextDoc();
|
||||||
writer.addDocument(doc);
|
writer.addDocument(doc);
|
||||||
long newRamSize = writer.ramSizeInBytes();
|
long newRamSize = writer.ramBytesUsed();
|
||||||
if (newRamSize != ramSize) {
|
if (newRamSize != ramSize) {
|
||||||
ramSize = newRamSize;
|
ramSize = newRamSize;
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user